summaryrefslogtreecommitdiff
path: root/compiler/optimizing
diff options
context:
space:
mode:
Diffstat (limited to 'compiler/optimizing')
-rw-r--r--compiler/optimizing/register_allocator_graph_color.cc1373
-rw-r--r--compiler/optimizing/register_allocator_graph_color.h60
-rw-r--r--compiler/optimizing/ssa_liveness_analysis.h4
3 files changed, 1209 insertions, 228 deletions
diff --git a/compiler/optimizing/register_allocator_graph_color.cc b/compiler/optimizing/register_allocator_graph_color.cc
index 79ca5a0d86..cfdb41ab62 100644
--- a/compiler/optimizing/register_allocator_graph_color.cc
+++ b/compiler/optimizing/register_allocator_graph_color.cc
@@ -37,6 +37,165 @@ static constexpr size_t kMaxNumRegs = 32;
// intervals are split when coloring fails.
static constexpr size_t kMaxGraphColoringAttemptsDebug = 100;
+// We always want to avoid spilling inside loops.
+static constexpr size_t kLoopSpillWeightMultiplier = 10;
+
+// If we avoid moves in single jump blocks, we can avoid jumps to jumps.
+static constexpr size_t kSingleJumpBlockWeightMultiplier = 2;
+
+// We avoid moves in blocks that dominate the exit block, since these blocks will
+// be executed on every path through the method.
+static constexpr size_t kDominatesExitBlockWeightMultiplier = 2;
+
+enum class CoalesceKind {
+ kAdjacentSibling, // Prevents moves at interval split points.
+ kFixedOutputSibling, // Prevents moves from a fixed output location.
+ kFixedInput, // Prevents moves into a fixed input location.
+ kNonlinearControlFlow, // Prevents moves between blocks.
+ kPhi, // Prevents phi resolution moves.
+ kFirstInput, // Prevents a single input move.
+ kAnyInput, // May lead to better instruction selection / smaller encodings.
+};
+
+std::ostream& operator<<(std::ostream& os, const CoalesceKind& kind) {
+ return os << static_cast<typename std::underlying_type<CoalesceKind>::type>(kind);
+}
+
+static size_t LoopDepthAt(HBasicBlock* block) {
+ HLoopInformation* loop_info = block->GetLoopInformation();
+ size_t depth = 0;
+ while (loop_info != nullptr) {
+ ++depth;
+ loop_info = loop_info->GetPreHeader()->GetLoopInformation();
+ }
+ return depth;
+}
+
+// Return the runtime cost of inserting a move instruction at the specified location.
+static size_t CostForMoveAt(size_t position, const SsaLivenessAnalysis& liveness) {
+ HBasicBlock* block = liveness.GetBlockFromPosition(position / 2);
+ DCHECK(block != nullptr);
+ size_t cost = 1;
+ if (block->IsSingleJump()) {
+ cost *= kSingleJumpBlockWeightMultiplier;
+ }
+ if (block->Dominates(block->GetGraph()->GetExitBlock())) {
+ cost *= kDominatesExitBlockWeightMultiplier;
+ }
+ for (size_t loop_depth = LoopDepthAt(block); loop_depth > 0; --loop_depth) {
+ cost *= kLoopSpillWeightMultiplier;
+ }
+ return cost;
+}
+
+// In general, we estimate coalesce priority by whether it will definitely avoid a move,
+// and by how likely it is to create an interference graph that's harder to color.
+static size_t ComputeCoalescePriority(CoalesceKind kind,
+ size_t position,
+ const SsaLivenessAnalysis& liveness) {
+ if (kind == CoalesceKind::kAnyInput) {
+ // This type of coalescing can affect instruction selection, but not moves, so we
+ // give it the lowest priority.
+ return 0;
+ } else {
+ return CostForMoveAt(position, liveness);
+ }
+}
+
+enum class CoalesceStage {
+ kWorklist, // Currently in the iterative coalescing worklist.
+ kActive, // Not in a worklist, but could be considered again during iterative coalescing.
+ kInactive, // No longer considered until last-chance coalescing.
+ kDefunct, // Either the two nodes interfere, or have already been coalesced.
+};
+
+std::ostream& operator<<(std::ostream& os, const CoalesceStage& stage) {
+ return os << static_cast<typename std::underlying_type<CoalesceStage>::type>(stage);
+}
+
+// Represents a coalesce opportunity between two nodes.
+struct CoalesceOpportunity : public ArenaObject<kArenaAllocRegisterAllocator> {
+ CoalesceOpportunity(InterferenceNode* a,
+ InterferenceNode* b,
+ CoalesceKind kind,
+ size_t position,
+ const SsaLivenessAnalysis& liveness)
+ : node_a(a),
+ node_b(b),
+ stage(CoalesceStage::kWorklist),
+ priority(ComputeCoalescePriority(kind, position, liveness)) {}
+
+ // Compare two coalesce opportunities based on their priority.
+ // Return true if lhs has a lower priority than that of rhs.
+ static bool CmpPriority(const CoalesceOpportunity* lhs,
+ const CoalesceOpportunity* rhs) {
+ return lhs->priority < rhs->priority;
+ }
+
+ InterferenceNode* const node_a;
+ InterferenceNode* const node_b;
+
+ // The current stage of this coalesce opportunity, indicating whether it is in a worklist,
+ // and whether it should still be considered.
+ CoalesceStage stage;
+
+ // The priority of this coalesce opportunity, based on heuristics.
+ const size_t priority;
+};
+
+enum class NodeStage {
+ kInitial, // Uninitialized.
+ kPrecolored, // Marks fixed nodes.
+ kSafepoint, // Marks safepoint nodes.
+ kPrunable, // Marks uncolored nodes in the interference graph.
+ kSimplifyWorklist, // Marks non-move-related nodes with degree less than the number of registers.
+ kFreezeWorklist, // Marks move-related nodes with degree less than the number of registers.
+ kSpillWorklist, // Marks nodes with degree greater or equal to the number of registers.
+ kPruned // Marks nodes already pruned from the interference graph.
+};
+
+std::ostream& operator<<(std::ostream& os, const NodeStage& stage) {
+ return os << static_cast<typename std::underlying_type<NodeStage>::type>(stage);
+}
+
+// Returns the estimated cost of spilling a particular live interval.
+static float ComputeSpillWeight(LiveInterval* interval, const SsaLivenessAnalysis& liveness) {
+ if (interval->HasRegister()) {
+ // Intervals with a fixed register cannot be spilled.
+ return std::numeric_limits<float>::min();
+ }
+
+ size_t length = interval->GetLength();
+ if (length == 1) {
+ // Tiny intervals should have maximum priority, since they cannot be split any further.
+ return std::numeric_limits<float>::max();
+ }
+
+ size_t use_weight = 0;
+ if (interval->GetDefinedBy() != nullptr && interval->DefinitionRequiresRegister()) {
+ // Cost for spilling at a register definition point.
+ use_weight += CostForMoveAt(interval->GetStart() + 1, liveness);
+ }
+
+ UsePosition* use = interval->GetFirstUse();
+ while (use != nullptr && use->GetPosition() <= interval->GetStart()) {
+ // Skip uses before the start of this live interval.
+ use = use->GetNext();
+ }
+
+ while (use != nullptr && use->GetPosition() <= interval->GetEnd()) {
+ if (use->GetUser() != nullptr && use->RequiresRegister()) {
+ // Cost for spilling at a register use point.
+ use_weight += CostForMoveAt(use->GetUser()->GetLifetimePosition() - 1, liveness);
+ }
+ use = use->GetNext();
+ }
+
+ // We divide by the length of the interval because we want to prioritize
+ // short intervals; we do not benefit much if we split them further.
+ return static_cast<float>(use_weight) / static_cast<float>(length);
+}
+
// Interference nodes make up the interference graph, which is the primary data structure in
// graph coloring register allocation. Each node represents a single live interval, and contains
// a set of adjacent nodes corresponding to intervals overlapping with its own. To save memory,
@@ -58,84 +217,320 @@ static constexpr size_t kMaxGraphColoringAttemptsDebug = 100;
// and thus whether it is safe to prune it from the interference graph early on.
class InterferenceNode : public ArenaObject<kArenaAllocRegisterAllocator> {
public:
- InterferenceNode(ArenaAllocator* allocator, LiveInterval* interval, size_t id)
- : interval_(interval),
- adjacent_nodes_(CmpPtr, allocator->Adapter(kArenaAllocRegisterAllocator)),
- out_degree_(0),
- id_(id) {}
-
- // Used to maintain determinism when storing InterferenceNode pointers in sets.
- static bool CmpPtr(const InterferenceNode* lhs, const InterferenceNode* rhs) {
- return lhs->id_ < rhs->id_;
+ InterferenceNode(ArenaAllocator* allocator,
+ LiveInterval* interval,
+ const SsaLivenessAnalysis& liveness)
+ : stage(NodeStage::kInitial),
+ interval_(interval),
+ adjacent_nodes_(allocator->Adapter(kArenaAllocRegisterAllocator)),
+ coalesce_opportunities_(allocator->Adapter(kArenaAllocRegisterAllocator)),
+ out_degree_(interval->HasRegister() ? std::numeric_limits<size_t>::max() : 0),
+ alias_(this),
+ spill_weight_(ComputeSpillWeight(interval, liveness)),
+ requires_color_(interval->RequiresRegister()) {
+ DCHECK(!interval->IsHighInterval()) << "Pair nodes should be represented by the low interval";
}
- void AddInterference(InterferenceNode* other) {
- if (adjacent_nodes_.insert(other).second) {
+ void AddInterference(InterferenceNode* other, bool guaranteed_not_interfering_yet) {
+ DCHECK(!IsPrecolored()) << "To save memory, fixed nodes should not have outgoing interferences";
+ DCHECK_NE(this, other) << "Should not create self loops in the interference graph";
+ DCHECK_EQ(this, alias_) << "Should not add interferences to a node that aliases another";
+ DCHECK_NE(stage, NodeStage::kPruned);
+ DCHECK_NE(other->stage, NodeStage::kPruned);
+ if (guaranteed_not_interfering_yet) {
+ DCHECK(std::find(adjacent_nodes_.begin(), adjacent_nodes_.end(), other)
+ == adjacent_nodes_.end());
+ adjacent_nodes_.push_back(other);
out_degree_ += EdgeWeightWith(other);
+ } else {
+ auto it = std::find(adjacent_nodes_.begin(), adjacent_nodes_.end(), other);
+ if (it == adjacent_nodes_.end()) {
+ adjacent_nodes_.push_back(other);
+ out_degree_ += EdgeWeightWith(other);
+ }
}
}
void RemoveInterference(InterferenceNode* other) {
- if (adjacent_nodes_.erase(other) > 0) {
+ DCHECK_EQ(this, alias_) << "Should not remove interferences from a coalesced node";
+ DCHECK_EQ(other->stage, NodeStage::kPruned) << "Should only remove interferences when pruning";
+ auto it = std::find(adjacent_nodes_.begin(), adjacent_nodes_.end(), other);
+ if (it != adjacent_nodes_.end()) {
+ adjacent_nodes_.erase(it);
out_degree_ -= EdgeWeightWith(other);
}
}
bool ContainsInterference(InterferenceNode* other) const {
- return adjacent_nodes_.count(other) > 0;
+ DCHECK(!IsPrecolored()) << "Should not query fixed nodes for interferences";
+ DCHECK_EQ(this, alias_) << "Should not query a coalesced node for interferences";
+ auto it = std::find(adjacent_nodes_.begin(), adjacent_nodes_.end(), other);
+ return it != adjacent_nodes_.end();
}
LiveInterval* GetInterval() const {
return interval_;
}
- const ArenaSet<InterferenceNode*, decltype(&CmpPtr)>& GetAdjacentNodes() const {
+ const ArenaVector<InterferenceNode*>& GetAdjacentNodes() const {
return adjacent_nodes_;
}
size_t GetOutDegree() const {
+ // Pre-colored nodes have infinite degree.
+ DCHECK(!IsPrecolored() || out_degree_ == std::numeric_limits<size_t>::max());
return out_degree_;
}
- size_t GetId() const {
- return id_;
+ void AddCoalesceOpportunity(CoalesceOpportunity* opportunity) {
+ coalesce_opportunities_.push_back(opportunity);
+ }
+
+ void ClearCoalesceOpportunities() {
+ coalesce_opportunities_.clear();
+ }
+
+ bool IsMoveRelated() const {
+ for (CoalesceOpportunity* opportunity : coalesce_opportunities_) {
+ if (opportunity->stage == CoalesceStage::kWorklist ||
+ opportunity->stage == CoalesceStage::kActive) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ // Return whether this node already has a color.
+ // Used to find fixed nodes in the interference graph before coloring.
+ bool IsPrecolored() const {
+ return interval_->HasRegister();
+ }
+
+ bool IsPair() const {
+ return interval_->HasHighInterval();
+ }
+
+ void SetAlias(InterferenceNode* rep) {
+ DCHECK_NE(rep->stage, NodeStage::kPruned);
+ DCHECK_EQ(this, alias_) << "Should only set a node's alias once";
+ alias_ = rep;
+ }
+
+ InterferenceNode* GetAlias() {
+ if (alias_ != this) {
+ // Recurse in order to flatten tree of alias pointers.
+ alias_ = alias_->GetAlias();
+ }
+ return alias_;
+ }
+
+ const ArenaVector<CoalesceOpportunity*>& GetCoalesceOpportunities() const {
+ return coalesce_opportunities_;
+ }
+
+ float GetSpillWeight() const {
+ return spill_weight_;
+ }
+
+ bool RequiresColor() const {
+ return requires_color_;
}
- private:
// We give extra weight to edges adjacent to pair nodes. See the general comment on the
// interference graph above.
- size_t EdgeWeightWith(InterferenceNode* other) const {
- return (interval_->HasHighInterval() || other->interval_->HasHighInterval()) ? 2 : 1;
+ size_t EdgeWeightWith(const InterferenceNode* other) const {
+ return (IsPair() || other->IsPair()) ? 2 : 1;
}
+ // The current stage of this node, indicating which worklist it belongs to.
+ NodeStage stage;
+
+ private:
// The live interval that this node represents.
LiveInterval* const interval_;
// All nodes interfering with this one.
- // TODO: There is potential to use a cheaper data structure here, especially since
- // adjacency sets will usually be small.
- ArenaSet<InterferenceNode*, decltype(&CmpPtr)> adjacent_nodes_;
+ // We use an unsorted vector as a set, since a tree or hash set is too heavy for the
+ // set sizes that we encounter. Using a vector leads to much better performance.
+ ArenaVector<InterferenceNode*> adjacent_nodes_;
+
+ // Interference nodes that this node should be coalesced with to reduce moves.
+ ArenaVector<CoalesceOpportunity*> coalesce_opportunities_;
// The maximum number of colors with which this node could interfere. This could be more than
// the number of adjacent nodes if this is a pair node, or if some adjacent nodes are pair nodes.
// We use "out" degree because incoming edges come from nodes already pruned from the graph,
// and do not affect the coloring of this node.
+ // Pre-colored nodes are treated as having infinite degree.
size_t out_degree_;
- // A unique identifier for this node, used to maintain determinism when storing
- // interference nodes in sets.
- const size_t id_;
+ // The node representing this node in the interference graph.
+ // Initially set to `this`, and only changed if this node is coalesced into another.
+ InterferenceNode* alias_;
- // TODO: We could cache the result of interval_->RequiresRegister(), since it
- // will not change for the lifetime of this node. (Currently, RequiresRegister() requires
- // iterating through all uses of a live interval.)
+ // The cost of splitting and spilling this interval to the stack.
+ // Nodes with a higher spill weight should be prioritized when assigning registers.
+ // This is essentially based on use density and location; short intervals with many uses inside
+ // deeply nested loops have a high spill weight.
+ const float spill_weight_;
+
+ const bool requires_color_;
DISALLOW_COPY_AND_ASSIGN(InterferenceNode);
};
+// The order in which we color nodes is important. To guarantee forward progress,
+// we prioritize intervals that require registers, and after that we prioritize
+// short intervals. That way, if we fail to color a node, it either won't require a
+// register, or it will be a long interval that can be split in order to make the
+// interference graph sparser.
+// To improve code quality, we prioritize intervals used frequently in deeply nested loops.
+// (This metric is secondary to the forward progress requirements above.)
+// TODO: May also want to consider:
+// - Constants (since they can be rematerialized)
+// - Allocated spill slots
+static bool HasGreaterNodePriority(const InterferenceNode* lhs,
+ const InterferenceNode* rhs) {
+ // (1) Prioritize the node that requires a color.
+ if (lhs->RequiresColor() != rhs->RequiresColor()) {
+ return lhs->RequiresColor();
+ }
+
+ // (2) Prioritize the interval that has a higher spill weight.
+ return lhs->GetSpillWeight() > rhs->GetSpillWeight();
+}
+
+// A ColoringIteration holds the many data structures needed for a single graph coloring attempt,
+// and provides methods for each phase of the attempt.
+class ColoringIteration {
+ public:
+ ColoringIteration(RegisterAllocatorGraphColor* register_allocator,
+ ArenaAllocator* allocator,
+ bool processing_core_regs,
+ size_t num_regs)
+ : register_allocator_(register_allocator),
+ allocator_(allocator),
+ processing_core_regs_(processing_core_regs),
+ num_regs_(num_regs),
+ interval_node_map_(allocator->Adapter(kArenaAllocRegisterAllocator)),
+ prunable_nodes_(allocator->Adapter(kArenaAllocRegisterAllocator)),
+ pruned_nodes_(allocator->Adapter(kArenaAllocRegisterAllocator)),
+ simplify_worklist_(allocator->Adapter(kArenaAllocRegisterAllocator)),
+ freeze_worklist_(allocator->Adapter(kArenaAllocRegisterAllocator)),
+ spill_worklist_(HasGreaterNodePriority, allocator->Adapter(kArenaAllocRegisterAllocator)),
+ coalesce_worklist_(CoalesceOpportunity::CmpPriority,
+ allocator->Adapter(kArenaAllocRegisterAllocator)) {}
+
+ // Use the intervals collected from instructions to construct an
+ // interference graph mapping intervals to adjacency lists.
+ // Also, collect synthesized safepoint nodes, used to keep
+ // track of live intervals across safepoints.
+ // TODO: Should build safepoints elsewhere.
+ void BuildInterferenceGraph(const ArenaVector<LiveInterval*>& intervals,
+ const ArenaVector<InterferenceNode*>& physical_nodes,
+ ArenaVector<InterferenceNode*>* safepoints);
+
+ // Add coalesce opportunities to interference nodes.
+ void FindCoalesceOpportunities();
+
+ // Prune nodes from the interference graph to be colored later. Build
+ // a stack (pruned_nodes) containing these intervals in an order determined
+ // by various heuristics.
+ void PruneInterferenceGraph();
+
+ // Process pruned_intervals_ to color the interference graph, spilling when
+ // necessary. Returns true if successful. Else, some intervals have been
+ // split, and the interference graph should be rebuilt for another attempt.
+ bool ColorInterferenceGraph();
+
+ // Return prunable nodes.
+ // The register allocator will need to access prunable nodes after coloring
+ // in order to tell the code generator which registers have been assigned.
+ const ArenaVector<InterferenceNode*>& GetPrunableNodes() const {
+ return prunable_nodes_;
+ }
+
+ private:
+ // Create a coalesce opportunity between two nodes.
+ void CreateCoalesceOpportunity(InterferenceNode* a,
+ InterferenceNode* b,
+ CoalesceKind kind,
+ size_t position);
+
+ // Add an edge in the interference graph, if valid.
+ // Note that `guaranteed_not_interfering_yet` is used to optimize adjacency set insertion
+ // when possible.
+ void AddPotentialInterference(InterferenceNode* from,
+ InterferenceNode* to,
+ bool guaranteed_not_interfering_yet,
+ bool both_directions = true);
+
+ // Invalidate all coalesce opportunities this node has, so that it (and possibly its neighbors)
+ // may be pruned from the interference graph.
+ void FreezeMoves(InterferenceNode* node);
+
+ // Prune a node from the interference graph, updating worklists if necessary.
+ void PruneNode(InterferenceNode* node);
+
+ // Add coalesce opportunities associated with this node to the coalesce worklist.
+ void EnableCoalesceOpportunities(InterferenceNode* node);
+
+ // If needed, from `node` from the freeze worklist to the simplify worklist.
+ void CheckTransitionFromFreezeWorklist(InterferenceNode* node);
+
+ // Return true if `into` is colored, and `from` can be coalesced with `into` conservatively.
+ bool PrecoloredHeuristic(InterferenceNode* from, InterferenceNode* into);
+
+ // Return true if `from` and `into` are uncolored, and can be coalesced conservatively.
+ bool UncoloredHeuristic(InterferenceNode* from, InterferenceNode* into);
+
+ void Coalesce(CoalesceOpportunity* opportunity);
+
+ // Merge `from` into `into` in the interference graph.
+ void Combine(InterferenceNode* from, InterferenceNode* into);
+
+ // A reference to the register allocator instance,
+ // needed to split intervals and assign spill slots.
+ RegisterAllocatorGraphColor* register_allocator_;
+
+ // An arena allocator used for a single graph coloring attempt.
+ ArenaAllocator* allocator_;
+
+ const bool processing_core_regs_;
+
+ const size_t num_regs_;
+
+ // A map from live intervals to interference nodes.
+ ArenaHashMap<LiveInterval*, InterferenceNode*> interval_node_map_;
+
+ // Uncolored nodes that should be pruned from the interference graph.
+ ArenaVector<InterferenceNode*> prunable_nodes_;
+
+ // A stack of nodes pruned from the interference graph, waiting to be pruned.
+ ArenaStdStack<InterferenceNode*> pruned_nodes_;
+
+ // A queue containing low degree, non-move-related nodes that can pruned immediately.
+ ArenaDeque<InterferenceNode*> simplify_worklist_;
+
+ // A queue containing low degree, move-related nodes.
+ ArenaDeque<InterferenceNode*> freeze_worklist_;
+
+ // A queue containing high degree nodes.
+ // If we have to prune from the spill worklist, we cannot guarantee
+ // the pruned node a color, so we order the worklist by priority.
+ ArenaPriorityQueue<InterferenceNode*, decltype(&HasGreaterNodePriority)> spill_worklist_;
+
+ // A queue containing coalesce opportunities.
+ // We order the coalesce worklist by priority, since some coalesce opportunities (e.g., those
+ // inside of loops) are more important than others.
+ ArenaPriorityQueue<CoalesceOpportunity*,
+ decltype(&CoalesceOpportunity::CmpPriority)> coalesce_worklist_;
+
+ DISALLOW_COPY_AND_ASSIGN(ColoringIteration);
+};
+
static bool IsCoreInterval(LiveInterval* interval) {
- return interval->GetType() != Primitive::kPrimFloat
- && interval->GetType() != Primitive::kPrimDouble;
+ return !Primitive::IsFloatingPointType(interval->GetType());
}
static size_t ComputeReservedArtMethodSlots(const CodeGenerator& codegen) {
@@ -144,14 +539,16 @@ static size_t ComputeReservedArtMethodSlots(const CodeGenerator& codegen) {
RegisterAllocatorGraphColor::RegisterAllocatorGraphColor(ArenaAllocator* allocator,
CodeGenerator* codegen,
- const SsaLivenessAnalysis& liveness)
+ const SsaLivenessAnalysis& liveness,
+ bool iterative_move_coalescing)
: RegisterAllocator(allocator, codegen, liveness),
+ iterative_move_coalescing_(iterative_move_coalescing),
core_intervals_(allocator->Adapter(kArenaAllocRegisterAllocator)),
fp_intervals_(allocator->Adapter(kArenaAllocRegisterAllocator)),
temp_intervals_(allocator->Adapter(kArenaAllocRegisterAllocator)),
safepoints_(allocator->Adapter(kArenaAllocRegisterAllocator)),
- physical_core_intervals_(allocator->Adapter(kArenaAllocRegisterAllocator)),
- physical_fp_intervals_(allocator->Adapter(kArenaAllocRegisterAllocator)),
+ physical_core_nodes_(allocator->Adapter(kArenaAllocRegisterAllocator)),
+ physical_fp_nodes_(allocator->Adapter(kArenaAllocRegisterAllocator)),
int_spill_slot_counter_(0),
double_spill_slot_counter_(0),
float_spill_slot_counter_(0),
@@ -162,17 +559,18 @@ RegisterAllocatorGraphColor::RegisterAllocatorGraphColor(ArenaAllocator* allocat
number_of_globally_blocked_core_regs_(0),
number_of_globally_blocked_fp_regs_(0),
max_safepoint_live_core_regs_(0),
- max_safepoint_live_fp_regs_(0),
- coloring_attempt_allocator_(nullptr) {
+ max_safepoint_live_fp_regs_(0) {
// Before we ask for blocked registers, set them up in the code generator.
codegen->SetupBlockedRegisters();
// Initialize physical core register live intervals and blocked registers.
// This includes globally blocked registers, such as the stack pointer.
- physical_core_intervals_.resize(codegen->GetNumberOfCoreRegisters(), nullptr);
- for (size_t i = 0; i < codegen->GetNumberOfCoreRegisters(); ++i) {
+ physical_core_nodes_.resize(codegen_->GetNumberOfCoreRegisters(), nullptr);
+ for (size_t i = 0; i < codegen_->GetNumberOfCoreRegisters(); ++i) {
LiveInterval* interval = LiveInterval::MakeFixedInterval(allocator_, i, Primitive::kPrimInt);
- physical_core_intervals_[i] = interval;
+ physical_core_nodes_[i] =
+ new (allocator_) InterferenceNode(allocator_, interval, liveness);
+ physical_core_nodes_[i]->stage = NodeStage::kPrecolored;
core_intervals_.push_back(interval);
if (codegen_->IsBlockedCoreRegister(i)) {
++number_of_globally_blocked_core_regs_;
@@ -180,10 +578,12 @@ RegisterAllocatorGraphColor::RegisterAllocatorGraphColor(ArenaAllocator* allocat
}
}
// Initialize physical floating point register live intervals and blocked registers.
- physical_fp_intervals_.resize(codegen->GetNumberOfFloatingPointRegisters(), nullptr);
- for (size_t i = 0; i < codegen->GetNumberOfFloatingPointRegisters(); ++i) {
+ physical_fp_nodes_.resize(codegen_->GetNumberOfFloatingPointRegisters(), nullptr);
+ for (size_t i = 0; i < codegen_->GetNumberOfFloatingPointRegisters(); ++i) {
LiveInterval* interval = LiveInterval::MakeFixedInterval(allocator_, i, Primitive::kPrimFloat);
- physical_fp_intervals_[i] = interval;
+ physical_fp_nodes_[i] =
+ new (allocator_) InterferenceNode(allocator_, interval, liveness);
+ physical_fp_nodes_[i]->stage = NodeStage::kPrecolored;
fp_intervals_.push_back(interval);
if (codegen_->IsBlockedFloatingPointRegister(i)) {
++number_of_globally_blocked_fp_regs_;
@@ -213,24 +613,44 @@ void RegisterAllocatorGraphColor::AllocateRegisters() {
<< "which could be caused by prioritizing the wrong live intervals. (Short intervals "
<< "should be prioritized over long ones, because they cannot be split further.)";
- // Reset the allocator for the next coloring attempt.
+ // Many data structures are cleared between graph coloring attempts, so we reduce
+ // total memory usage by using a new arena allocator for each attempt.
ArenaAllocator coloring_attempt_allocator(allocator_->GetArenaPool());
- coloring_attempt_allocator_ = &coloring_attempt_allocator;
+ ColoringIteration iteration(this,
+ &coloring_attempt_allocator,
+ processing_core_regs,
+ num_registers);
- // (2) Build the interference graph.
- ArenaVector<InterferenceNode*> prunable_nodes(
- coloring_attempt_allocator_->Adapter(kArenaAllocRegisterAllocator));
+ // (2) Build the interference graph. Also gather safepoints.
ArenaVector<InterferenceNode*> safepoints(
- coloring_attempt_allocator_->Adapter(kArenaAllocRegisterAllocator));
- BuildInterferenceGraph(intervals, &prunable_nodes, &safepoints);
+ coloring_attempt_allocator.Adapter(kArenaAllocRegisterAllocator));
+ ArenaVector<InterferenceNode*>& physical_nodes = processing_core_regs
+ ? physical_core_nodes_
+ : physical_fp_nodes_;
+ iteration.BuildInterferenceGraph(intervals, physical_nodes, &safepoints);
+
+ // (3) Add coalesce opportunities.
+ // If we have tried coloring the graph a suspiciously high number of times, give
+ // up on move coalescing, just in case the coalescing heuristics are not conservative.
+ // (This situation will be caught if DCHECKs are turned on.)
+ if (iterative_move_coalescing_ && attempt <= kMaxGraphColoringAttemptsDebug) {
+ iteration.FindCoalesceOpportunities();
+ }
- // (3) Prune all uncolored nodes from interference graph.
- ArenaStdStack<InterferenceNode*> pruned_nodes(
- coloring_attempt_allocator_->Adapter(kArenaAllocRegisterAllocator));
- PruneInterferenceGraph(prunable_nodes, num_registers, &pruned_nodes);
+ // (4) Prune all uncolored nodes from interference graph.
+ iteration.PruneInterferenceGraph();
- // (4) Color pruned nodes based on interferences.
- bool successful = ColorInterferenceGraph(&pruned_nodes, num_registers);
+ // (5) Color pruned nodes based on interferences.
+ bool successful = iteration.ColorInterferenceGraph();
+
+ // We manually clear coalesce opportunities for physical nodes,
+ // since they persist across coloring attempts.
+ for (InterferenceNode* node : physical_core_nodes_) {
+ node->ClearCoalesceOpportunities();
+ }
+ for (InterferenceNode* node : physical_fp_nodes_) {
+ node->ClearCoalesceOpportunities();
+ }
if (successful) {
// Compute the maximum number of live registers across safepoints.
@@ -250,7 +670,7 @@ void RegisterAllocatorGraphColor::AllocateRegisters() {
// We only look at prunable_nodes because we already told the code generator about
// fixed intervals while processing instructions. We also ignore the fixed intervals
// placed at the top of catch blocks.
- for (InterferenceNode* node : prunable_nodes) {
+ for (InterferenceNode* node : iteration.GetPrunableNodes()) {
LiveInterval* interval = node->GetInterval();
if (interval->HasRegister()) {
Location low_reg = processing_core_regs
@@ -275,7 +695,7 @@ void RegisterAllocatorGraphColor::AllocateRegisters() {
} // while unsuccessful
} // for processing_core_instructions
- // (5) Resolve locations and deconstruct SSA form.
+ // (6) Resolve locations and deconstruct SSA form.
RegisterAllocationResolver(allocator_, codegen_, liveness_)
.Resolve(max_safepoint_live_core_regs_,
max_safepoint_live_fp_regs_,
@@ -304,11 +724,12 @@ bool RegisterAllocatorGraphColor::Validate(bool log_fatal_on_failure) {
}
}
- ArenaVector<LiveInterval*>& physical_intervals = processing_core_regs
- ? physical_core_intervals_
- : physical_fp_intervals_;
- for (LiveInterval* fixed : physical_intervals) {
- if (fixed->GetFirstRange() != nullptr) {
+ ArenaVector<InterferenceNode*>& physical_nodes = processing_core_regs
+ ? physical_core_nodes_
+ : physical_fp_nodes_;
+ for (InterferenceNode* fixed : physical_nodes) {
+ LiveInterval* interval = fixed->GetInterval();
+ if (interval->GetFirstRange() != nullptr) {
// Ideally we would check fixed ranges as well, but currently there are times when
// two fixed intervals for the same register will overlap. For example, a fixed input
// and a fixed output may sometimes share the same register, in which there will be two
@@ -358,7 +779,8 @@ void RegisterAllocatorGraphColor::ProcessInstructions() {
ProcessInstruction(phi_it.Current());
}
- if (block->IsCatchBlock() || (block->IsLoopHeader() && block->GetLoopInformation()->IsIrreducible())) {
+ if (block->IsCatchBlock()
+ || (block->IsLoopHeader() && block->GetLoopInformation()->IsIrreducible())) {
// By blocking all registers at the top of each catch block or irreducible loop, we force
// intervals belonging to the live-in set of the catch/header block to be spilled.
// TODO(ngeoffray): Phis in this block could be allocated in register.
@@ -435,7 +857,9 @@ void RegisterAllocatorGraphColor::CheckForFixedInputs(HInstruction* instruction)
// TODO: Ideally we would coalesce the physical register with the register
// allocated to the input value, but this can be tricky if, e.g., there
// could be multiple physical register uses of the same value at the
- // same instruction. Need to think about it more.
+ // same instruction. Furthermore, there's currently no distinction between
+ // fixed inputs to a call (which will be clobbered) and other fixed inputs (which
+ // may not be clobbered).
LocationSummary* locations = instruction->GetLocations();
size_t position = instruction->GetLifetimePosition();
for (size_t i = 0; i < locations->GetInputCount(); ++i) {
@@ -639,8 +1063,8 @@ void RegisterAllocatorGraphColor::BlockRegister(Location location,
DCHECK(location.IsRegister() || location.IsFpuRegister());
int reg = location.reg();
LiveInterval* interval = location.IsRegister()
- ? physical_core_intervals_[reg]
- : physical_fp_intervals_[reg];
+ ? physical_core_nodes_[reg]->GetInterval()
+ : physical_fp_nodes_[reg]->GetInterval();
DCHECK(interval->GetRegister() == reg);
bool blocked_by_codegen = location.IsRegister()
? codegen_->IsBlockedCoreRegister(reg)
@@ -666,28 +1090,105 @@ void RegisterAllocatorGraphColor::BlockRegisters(size_t start, size_t end, bool
}
}
-// Add an interference edge, but only if necessary.
-static void AddPotentialInterference(InterferenceNode* from, InterferenceNode* to) {
- if (from->GetInterval()->HasRegister()) {
+void ColoringIteration::AddPotentialInterference(InterferenceNode* from,
+ InterferenceNode* to,
+ bool guaranteed_not_interfering_yet,
+ bool both_directions) {
+ if (from->IsPrecolored()) {
// We save space by ignoring outgoing edges from fixed nodes.
} else if (to->GetInterval()->IsSlowPathSafepoint()) {
// Safepoint intervals are only there to count max live registers,
// so no need to give them incoming interference edges.
// This is also necessary for correctness, because we don't want nodes
// to remove themselves from safepoint adjacency sets when they're pruned.
+ } else if (to->IsPrecolored()) {
+ // It is important that only a single node represents a given fixed register in the
+ // interference graph. We retrieve that node here.
+ const ArenaVector<InterferenceNode*>& physical_nodes = to->GetInterval()->IsFloatingPoint()
+ ? register_allocator_->physical_fp_nodes_
+ : register_allocator_->physical_core_nodes_;
+ InterferenceNode* physical_node = physical_nodes[to->GetInterval()->GetRegister()];
+ from->AddInterference(physical_node, /*guaranteed_not_interfering_yet*/ false);
+ DCHECK_EQ(to->GetInterval()->GetRegister(), physical_node->GetInterval()->GetRegister());
+ DCHECK_EQ(to->GetAlias(), physical_node) << "Fixed nodes should alias the canonical fixed node";
+
+ // If a node interferes with a fixed pair node, the weight of the edge may
+ // be inaccurate after using the alias of the pair node, because the alias of the pair node
+ // is a singular node.
+ // We could make special pair fixed nodes, but that ends up being too conservative because
+ // a node could then interfere with both {r1} and {r1,r2}, leading to a degree of
+ // three rather than two.
+ // Instead, we explicitly add an interference with the high node of the fixed pair node.
+ // TODO: This is too conservative at time for pair nodes, but the fact that fixed pair intervals
+ // can be unaligned on x86 complicates things.
+ if (to->IsPair()) {
+ InterferenceNode* high_node =
+ physical_nodes[to->GetInterval()->GetHighInterval()->GetRegister()];
+ DCHECK_EQ(to->GetInterval()->GetHighInterval()->GetRegister(),
+ high_node->GetInterval()->GetRegister());
+ from->AddInterference(high_node, /*guaranteed_not_interfering_yet*/ false);
+ }
} else {
- from->AddInterference(to);
+ // Standard interference between two uncolored nodes.
+ from->AddInterference(to, guaranteed_not_interfering_yet);
+ }
+
+ if (both_directions) {
+ AddPotentialInterference(to, from, guaranteed_not_interfering_yet, /*both_directions*/ false);
}
}
-// TODO: See locations->OutputCanOverlapWithInputs(); we may want to consider
-// this when building the interference graph.
-void RegisterAllocatorGraphColor::BuildInterferenceGraph(
+// Returns true if `in_node` represents an input interval of `out_node`, and the output interval
+// is allowed to have the same register as the input interval.
+// TODO: Ideally we should just produce correct intervals in liveness analysis.
+// We would need to refactor the current live interval layout to do so, which is
+// no small task.
+static bool CheckInputOutputCanOverlap(InterferenceNode* in_node, InterferenceNode* out_node) {
+ LiveInterval* output_interval = out_node->GetInterval();
+ HInstruction* defined_by = output_interval->GetDefinedBy();
+ if (defined_by == nullptr) {
+ // This must not be a definition point.
+ return false;
+ }
+
+ LocationSummary* locations = defined_by->GetLocations();
+ if (locations->OutputCanOverlapWithInputs()) {
+ // This instruction does not allow the output to reuse a register from an input.
+ return false;
+ }
+
+ LiveInterval* input_interval = in_node->GetInterval();
+ LiveInterval* next_sibling = input_interval->GetNextSibling();
+ size_t def_position = defined_by->GetLifetimePosition();
+ size_t use_position = def_position + 1;
+ if (next_sibling != nullptr && next_sibling->GetStart() == use_position) {
+ // The next sibling starts at the use position, so reusing the input register in the output
+ // would clobber the input before it's moved into the sibling interval location.
+ return false;
+ }
+
+ if (!input_interval->IsDeadAt(use_position) && input_interval->CoversSlow(use_position)) {
+ // The input interval is live after the use position.
+ return false;
+ }
+
+ HInputsRef inputs = defined_by->GetInputs();
+ for (size_t i = 0; i < inputs.size(); ++i) {
+ if (inputs[i]->GetLiveInterval()->GetSiblingAt(def_position) == input_interval) {
+ DCHECK(input_interval->SameRegisterKind(*output_interval));
+ return true;
+ }
+ }
+
+ // The input interval was not an input for this instruction.
+ return false;
+}
+
+void ColoringIteration::BuildInterferenceGraph(
const ArenaVector<LiveInterval*>& intervals,
- ArenaVector<InterferenceNode*>* prunable_nodes,
+ const ArenaVector<InterferenceNode*>& physical_nodes,
ArenaVector<InterferenceNode*>* safepoints) {
- size_t interval_id_counter = 0;
-
+ DCHECK(interval_node_map_.Empty() && prunable_nodes_.empty());
// Build the interference graph efficiently by ordering range endpoints
// by position and doing a linear sweep to find interferences. (That is, we
// jump from endpoint to endpoint, maintaining a set of intervals live at each
@@ -701,21 +1202,34 @@ void RegisterAllocatorGraphColor::BuildInterferenceGraph(
// For simplicity, we create a tuple for each endpoint, and then sort the tuples.
// Tuple contents: (position, is_range_beginning, node).
ArenaVector<std::tuple<size_t, bool, InterferenceNode*>> range_endpoints(
- coloring_attempt_allocator_->Adapter(kArenaAllocRegisterAllocator));
+ allocator_->Adapter(kArenaAllocRegisterAllocator));
+
+ // We reserve plenty of space to avoid excessive copying.
+ range_endpoints.reserve(4 * prunable_nodes_.size());
+
for (LiveInterval* parent : intervals) {
for (LiveInterval* sibling = parent; sibling != nullptr; sibling = sibling->GetNextSibling()) {
LiveRange* range = sibling->GetFirstRange();
if (range != nullptr) {
- InterferenceNode* node = new (coloring_attempt_allocator_) InterferenceNode(
- coloring_attempt_allocator_, sibling, interval_id_counter++);
+ InterferenceNode* node = new (allocator_) InterferenceNode(
+ allocator_, sibling, register_allocator_->liveness_);
+ interval_node_map_.Insert(std::make_pair(sibling, node));
+
if (sibling->HasRegister()) {
- // Fixed nodes will never be pruned, so no need to keep track of them.
+ // Fixed nodes should alias the canonical node for the corresponding register.
+ node->stage = NodeStage::kPrecolored;
+ InterferenceNode* physical_node = physical_nodes[sibling->GetRegister()];
+ node->SetAlias(physical_node);
+ DCHECK_EQ(node->GetInterval()->GetRegister(),
+ physical_node->GetInterval()->GetRegister());
} else if (sibling->IsSlowPathSafepoint()) {
// Safepoint intervals are synthesized to count max live registers.
// They will be processed separately after coloring.
+ node->stage = NodeStage::kSafepoint;
safepoints->push_back(node);
} else {
- prunable_nodes->push_back(node);
+ node->stage = NodeStage::kPrunable;
+ prunable_nodes_.push_back(node);
}
while (range != nullptr) {
@@ -728,11 +1242,18 @@ void RegisterAllocatorGraphColor::BuildInterferenceGraph(
}
// Sort the endpoints.
- std::sort(range_endpoints.begin(), range_endpoints.end());
+ // We explicitly ignore the third entry of each tuple (the node pointer) in order
+ // to maintain determinism.
+ std::sort(range_endpoints.begin(), range_endpoints.end(),
+ [] (const std::tuple<size_t, bool, InterferenceNode*>& lhs,
+ const std::tuple<size_t, bool, InterferenceNode*>& rhs) {
+ return std::tie(std::get<0>(lhs), std::get<1>(lhs))
+ < std::tie(std::get<0>(rhs), std::get<1>(rhs));
+ });
// Nodes live at the current position in the linear sweep.
- ArenaSet<InterferenceNode*, decltype(&InterferenceNode::CmpPtr)> live(
- InterferenceNode::CmpPtr, coloring_attempt_allocator_->Adapter(kArenaAllocRegisterAllocator));
+ ArenaVector<InterferenceNode*> live(
+ allocator_->Adapter(kArenaAllocRegisterAllocator));
// Linear sweep. When we encounter the beginning of a range, we add the corresponding node to the
// live set. When we encounter the end of a range, we remove the corresponding node
@@ -740,131 +1261,505 @@ void RegisterAllocatorGraphColor::BuildInterferenceGraph(
for (auto it = range_endpoints.begin(); it != range_endpoints.end(); ++it) {
bool is_range_beginning;
InterferenceNode* node;
+ size_t position;
// Extract information from the tuple, including the node this tuple represents.
- std::tie(std::ignore, is_range_beginning, node) = *it;
+ std::tie(position, is_range_beginning, node) = *it;
if (is_range_beginning) {
+ bool guaranteed_not_interfering_yet = position == node->GetInterval()->GetStart();
for (InterferenceNode* conflicting : live) {
DCHECK_NE(node, conflicting);
- AddPotentialInterference(node, conflicting);
- AddPotentialInterference(conflicting, node);
+ if (CheckInputOutputCanOverlap(conflicting, node)) {
+ // We do not add an interference, because the instruction represented by `node` allows
+ // its output to share a register with an input, represented here by `conflicting`.
+ } else {
+ AddPotentialInterference(node, conflicting, guaranteed_not_interfering_yet);
+ }
}
- DCHECK_EQ(live.count(node), 0u);
- live.insert(node);
+ DCHECK(std::find(live.begin(), live.end(), node) == live.end());
+ live.push_back(node);
} else {
// End of range.
- DCHECK_EQ(live.count(node), 1u);
- live.erase(node);
+ auto live_it = std::find(live.begin(), live.end(), node);
+ DCHECK(live_it != live.end());
+ live.erase(live_it);
}
}
DCHECK(live.empty());
}
-// The order in which we color nodes is vital to both correctness (forward
-// progress) and code quality. Specifically, we must prioritize intervals
-// that require registers, and after that we must prioritize short intervals.
-// That way, if we fail to color a node, it either won't require a register,
-// or it will be a long interval that can be split in order to make the
-// interference graph sparser.
-// TODO: May also want to consider:
-// - Loop depth
-// - Constants (since they can be rematerialized)
-// - Allocated spill slots
-static bool GreaterNodePriority(const InterferenceNode* lhs,
- const InterferenceNode* rhs) {
- LiveInterval* lhs_interval = lhs->GetInterval();
- LiveInterval* rhs_interval = rhs->GetInterval();
+void ColoringIteration::CreateCoalesceOpportunity(InterferenceNode* a,
+ InterferenceNode* b,
+ CoalesceKind kind,
+ size_t position) {
+ DCHECK_EQ(a->IsPair(), b->IsPair())
+ << "Nodes of different memory widths should never be coalesced";
+ CoalesceOpportunity* opportunity =
+ new (allocator_) CoalesceOpportunity(a, b, kind, position, register_allocator_->liveness_);
+ a->AddCoalesceOpportunity(opportunity);
+ b->AddCoalesceOpportunity(opportunity);
+ coalesce_worklist_.push(opportunity);
+}
- // (1) Choose the interval that requires a register.
- if (lhs_interval->RequiresRegister() != rhs_interval->RequiresRegister()) {
- return lhs_interval->RequiresRegister();
- }
+// When looking for coalesce opportunities, we use the interval_node_map_ to find the node
+// corresponding to an interval. Note that not all intervals are in this map, notably the parents
+// of constants and stack arguments. (However, these interval should not be involved in coalesce
+// opportunities anyway, because they're not going to be in registers.)
+void ColoringIteration::FindCoalesceOpportunities() {
+ DCHECK(coalesce_worklist_.empty());
- // (2) Choose the interval that has a shorter life span.
- if (lhs_interval->GetLength() != rhs_interval->GetLength()) {
- return lhs_interval->GetLength() < rhs_interval->GetLength();
- }
+ for (InterferenceNode* node : prunable_nodes_) {
+ LiveInterval* interval = node->GetInterval();
+
+ // Coalesce siblings.
+ LiveInterval* next_sibling = interval->GetNextSibling();
+ if (next_sibling != nullptr && interval->GetEnd() == next_sibling->GetStart()) {
+ auto it = interval_node_map_.Find(next_sibling);
+ if (it != interval_node_map_.end()) {
+ InterferenceNode* sibling_node = it->second;
+ CreateCoalesceOpportunity(node,
+ sibling_node,
+ CoalesceKind::kAdjacentSibling,
+ interval->GetEnd());
+ }
+ }
+
+ // Coalesce fixed outputs with this interval if this interval is an adjacent sibling.
+ LiveInterval* parent = interval->GetParent();
+ if (parent->HasRegister()
+ && parent->GetNextSibling() == interval
+ && parent->GetEnd() == interval->GetStart()) {
+ auto it = interval_node_map_.Find(parent);
+ if (it != interval_node_map_.end()) {
+ InterferenceNode* parent_node = it->second;
+ CreateCoalesceOpportunity(node,
+ parent_node,
+ CoalesceKind::kFixedOutputSibling,
+ parent->GetEnd());
+ }
+ }
+
+ // Try to prevent moves across blocks.
+ // Note that this does not lead to many succeeding coalesce attempts, so could be removed
+ // if found to add to compile time.
+ const SsaLivenessAnalysis& liveness = register_allocator_->liveness_;
+ if (interval->IsSplit() && liveness.IsAtBlockBoundary(interval->GetStart() / 2)) {
+ // If the start of this interval is at a block boundary, we look at the
+ // location of the interval in blocks preceding the block this interval
+ // starts at. This can avoid a move between the two blocks.
+ HBasicBlock* block = liveness.GetBlockFromPosition(interval->GetStart() / 2);
+ for (HBasicBlock* predecessor : block->GetPredecessors()) {
+ size_t position = predecessor->GetLifetimeEnd() - 1;
+ LiveInterval* existing = interval->GetParent()->GetSiblingAt(position);
+ if (existing != nullptr) {
+ auto it = interval_node_map_.Find(existing);
+ if (it != interval_node_map_.end()) {
+ InterferenceNode* existing_node = it->second;
+ CreateCoalesceOpportunity(node,
+ existing_node,
+ CoalesceKind::kNonlinearControlFlow,
+ position);
+ }
+ }
+ }
+ }
+
+ // Coalesce phi inputs with the corresponding output.
+ HInstruction* defined_by = interval->GetDefinedBy();
+ if (defined_by != nullptr && defined_by->IsPhi()) {
+ const ArenaVector<HBasicBlock*>& predecessors = defined_by->GetBlock()->GetPredecessors();
+ HInputsRef inputs = defined_by->GetInputs();
+
+ for (size_t i = 0, e = inputs.size(); i < e; ++i) {
+ // We want the sibling at the end of the appropriate predecessor block.
+ size_t position = predecessors[i]->GetLifetimeEnd() - 1;
+ LiveInterval* input_interval = inputs[i]->GetLiveInterval()->GetSiblingAt(position);
+
+ auto it = interval_node_map_.Find(input_interval);
+ if (it != interval_node_map_.end()) {
+ InterferenceNode* input_node = it->second;
+ CreateCoalesceOpportunity(node, input_node, CoalesceKind::kPhi, position);
+ }
+ }
+ }
+
+ // Coalesce output with first input when policy is kSameAsFirstInput.
+ if (defined_by != nullptr) {
+ Location out = defined_by->GetLocations()->Out();
+ if (out.IsUnallocated() && out.GetPolicy() == Location::kSameAsFirstInput) {
+ LiveInterval* input_interval
+ = defined_by->InputAt(0)->GetLiveInterval()->GetSiblingAt(interval->GetStart() - 1);
+ // TODO: Could we consider lifetime holes here?
+ if (input_interval->GetEnd() == interval->GetStart()) {
+ auto it = interval_node_map_.Find(input_interval);
+ if (it != interval_node_map_.end()) {
+ InterferenceNode* input_node = it->second;
+ CreateCoalesceOpportunity(node,
+ input_node,
+ CoalesceKind::kFirstInput,
+ interval->GetStart());
+ }
+ }
+ }
+ }
+
+ // An interval that starts an instruction (that is, it is not split), may
+ // re-use the registers used by the inputs of that instruction, based on the
+ // location summary.
+ if (defined_by != nullptr) {
+ DCHECK(!interval->IsSplit());
+ LocationSummary* locations = defined_by->GetLocations();
+ if (!locations->OutputCanOverlapWithInputs()) {
+ HInputsRef inputs = defined_by->GetInputs();
+ for (size_t i = 0; i < inputs.size(); ++i) {
+ size_t def_point = defined_by->GetLifetimePosition();
+ // TODO: Getting the sibling at the def_point might not be quite what we want
+ // for fixed inputs, since the use will be *at* the def_point rather than after.
+ LiveInterval* input_interval = inputs[i]->GetLiveInterval()->GetSiblingAt(def_point);
+ if (input_interval != nullptr &&
+ input_interval->HasHighInterval() == interval->HasHighInterval()) {
+ auto it = interval_node_map_.Find(input_interval);
+ if (it != interval_node_map_.end()) {
+ InterferenceNode* input_node = it->second;
+ CreateCoalesceOpportunity(node,
+ input_node,
+ CoalesceKind::kAnyInput,
+ interval->GetStart());
+ }
+ }
+ }
+ }
+ }
+
+ // Try to prevent moves into fixed input locations.
+ UsePosition* use = interval->GetFirstUse();
+ for (; use != nullptr && use->GetPosition() <= interval->GetStart(); use = use->GetNext()) {
+ // Skip past uses before the start of this interval.
+ }
+ for (; use != nullptr && use->GetPosition() <= interval->GetEnd(); use = use->GetNext()) {
+ HInstruction* user = use->GetUser();
+ if (user == nullptr) {
+ // User may be null for certain intervals, such as temp intervals.
+ continue;
+ }
+ LocationSummary* locations = user->GetLocations();
+ Location input = locations->InAt(use->GetInputIndex());
+ if (input.IsRegister() || input.IsFpuRegister()) {
+ // TODO: Could try to handle pair interval too, but coalescing with fixed pair nodes
+ // is currently not supported.
+ InterferenceNode* fixed_node = input.IsRegister()
+ ? register_allocator_->physical_core_nodes_[input.reg()]
+ : register_allocator_->physical_fp_nodes_[input.reg()];
+ CreateCoalesceOpportunity(node,
+ fixed_node,
+ CoalesceKind::kFixedInput,
+ user->GetLifetimePosition());
+ }
+ }
+ } // for node in prunable_nodes
+}
- // (3) Just choose the interval based on a deterministic ordering.
- return InterferenceNode::CmpPtr(lhs, rhs);
+static bool IsLowDegreeNode(InterferenceNode* node, size_t num_regs) {
+ return node->GetOutDegree() < num_regs;
}
-void RegisterAllocatorGraphColor::PruneInterferenceGraph(
- const ArenaVector<InterferenceNode*>& prunable_nodes,
- size_t num_regs,
- ArenaStdStack<InterferenceNode*>* pruned_nodes) {
+static bool IsHighDegreeNode(InterferenceNode* node, size_t num_regs) {
+ return !IsLowDegreeNode(node, num_regs);
+}
+
+void ColoringIteration::PruneInterferenceGraph() {
+ DCHECK(pruned_nodes_.empty()
+ && simplify_worklist_.empty()
+ && freeze_worklist_.empty()
+ && spill_worklist_.empty());
// When pruning the graph, we refer to nodes with degree less than num_regs as low degree nodes,
// and all others as high degree nodes. The distinction is important: low degree nodes are
// guaranteed a color, while high degree nodes are not.
- // Low-degree nodes are guaranteed a color, so worklist order does not matter.
- ArenaDeque<InterferenceNode*> low_degree_worklist(
- coloring_attempt_allocator_->Adapter(kArenaAllocRegisterAllocator));
-
- // If we have to prune from the high-degree worklist, we cannot guarantee
- // the pruned node a color. So, we order the worklist by priority.
- ArenaSet<InterferenceNode*, decltype(&GreaterNodePriority)> high_degree_worklist(
- GreaterNodePriority, coloring_attempt_allocator_->Adapter(kArenaAllocRegisterAllocator));
-
- // Build worklists.
- for (InterferenceNode* node : prunable_nodes) {
- DCHECK(!node->GetInterval()->HasRegister())
- << "Fixed nodes should never be pruned";
- DCHECK(!node->GetInterval()->IsSlowPathSafepoint())
- << "Safepoint nodes should never be pruned";
- if (node->GetOutDegree() < num_regs) {
- low_degree_worklist.push_back(node);
- } else {
- high_degree_worklist.insert(node);
- }
- }
-
- // Helper function to prune an interval from the interference graph,
- // which includes updating the worklists.
- auto prune_node = [this,
- num_regs,
- &pruned_nodes,
- &low_degree_worklist,
- &high_degree_worklist] (InterferenceNode* node) {
- DCHECK(!node->GetInterval()->HasRegister());
- pruned_nodes->push(node);
- for (InterferenceNode* adjacent : node->GetAdjacentNodes()) {
- DCHECK(!adjacent->GetInterval()->IsSlowPathSafepoint())
- << "Nodes should never interfere with synthesized safepoint nodes";
- if (adjacent->GetInterval()->HasRegister()) {
- // No effect on pre-colored nodes; they're never pruned.
+ // Build worklists. Note that the coalesce worklist has already been
+ // filled by FindCoalesceOpportunities().
+ for (InterferenceNode* node : prunable_nodes_) {
+ DCHECK(!node->IsPrecolored()) << "Fixed nodes should never be pruned";
+ DCHECK(!node->GetInterval()->IsSlowPathSafepoint()) << "Safepoint nodes should never be pruned";
+ if (IsLowDegreeNode(node, num_regs_)) {
+ if (node->GetCoalesceOpportunities().empty()) {
+ // Simplify Worklist.
+ node->stage = NodeStage::kSimplifyWorklist;
+ simplify_worklist_.push_back(node);
} else {
- bool was_high_degree = adjacent->GetOutDegree() >= num_regs;
- DCHECK(adjacent->ContainsInterference(node))
- << "Missing incoming interference edge from non-fixed node";
- adjacent->RemoveInterference(node);
- if (was_high_degree && adjacent->GetOutDegree() < num_regs) {
- // This is a transition from high degree to low degree.
- DCHECK_EQ(high_degree_worklist.count(adjacent), 1u);
- high_degree_worklist.erase(adjacent);
- low_degree_worklist.push_back(adjacent);
- }
+ // Freeze Worklist.
+ node->stage = NodeStage::kFreezeWorklist;
+ freeze_worklist_.push_back(node);
}
+ } else {
+ // Spill worklist.
+ node->stage = NodeStage::kSpillWorklist;
+ spill_worklist_.push(node);
}
- };
+ }
// Prune graph.
- while (!low_degree_worklist.empty() || !high_degree_worklist.empty()) {
- while (!low_degree_worklist.empty()) {
- InterferenceNode* node = low_degree_worklist.front();
- // TODO: pop_back() should work as well, but it doesn't; we get a
+ // Note that we do not remove a node from its current worklist if it moves to another, so it may
+ // be in multiple worklists at once; the node's `phase` says which worklist it is really in.
+ while (true) {
+ if (!simplify_worklist_.empty()) {
+ // Prune low-degree nodes.
+ // TODO: pop_back() should work as well, but it didn't; we get a
// failed check while pruning. We should look into this.
- low_degree_worklist.pop_front();
- prune_node(node);
- }
- if (!high_degree_worklist.empty()) {
- // We prune the lowest-priority node, because pruning a node earlier
+ InterferenceNode* node = simplify_worklist_.front();
+ simplify_worklist_.pop_front();
+ DCHECK_EQ(node->stage, NodeStage::kSimplifyWorklist) << "Cannot move from simplify list";
+ DCHECK_LT(node->GetOutDegree(), num_regs_) << "Nodes in simplify list should be low degree";
+ DCHECK(!node->IsMoveRelated()) << "Nodes in simplify list should not be move related";
+ PruneNode(node);
+ } else if (!coalesce_worklist_.empty()) {
+ // Coalesce.
+ CoalesceOpportunity* opportunity = coalesce_worklist_.top();
+ coalesce_worklist_.pop();
+ if (opportunity->stage == CoalesceStage::kWorklist) {
+ Coalesce(opportunity);
+ }
+ } else if (!freeze_worklist_.empty()) {
+ // Freeze moves and prune a low-degree move-related node.
+ InterferenceNode* node = freeze_worklist_.front();
+ freeze_worklist_.pop_front();
+ if (node->stage == NodeStage::kFreezeWorklist) {
+ DCHECK_LT(node->GetOutDegree(), num_regs_) << "Nodes in freeze list should be low degree";
+ DCHECK(node->IsMoveRelated()) << "Nodes in freeze list should be move related";
+ FreezeMoves(node);
+ PruneNode(node);
+ }
+ } else if (!spill_worklist_.empty()) {
+ // We spill the lowest-priority node, because pruning a node earlier
// gives it a higher chance of being spilled.
- InterferenceNode* node = *high_degree_worklist.rbegin();
- high_degree_worklist.erase(node);
- prune_node(node);
+ InterferenceNode* node = spill_worklist_.top();
+ spill_worklist_.pop();
+ if (node->stage == NodeStage::kSpillWorklist) {
+ DCHECK_GE(node->GetOutDegree(), num_regs_) << "Nodes in spill list should be high degree";
+ FreezeMoves(node);
+ PruneNode(node);
+ }
+ } else {
+ // Pruning complete.
+ break;
+ }
+ }
+ DCHECK_EQ(prunable_nodes_.size(), pruned_nodes_.size());
+}
+
+void ColoringIteration::EnableCoalesceOpportunities(InterferenceNode* node) {
+ for (CoalesceOpportunity* opportunity : node->GetCoalesceOpportunities()) {
+ if (opportunity->stage == CoalesceStage::kActive) {
+ opportunity->stage = CoalesceStage::kWorklist;
+ coalesce_worklist_.push(opportunity);
+ }
+ }
+}
+
+void ColoringIteration::PruneNode(InterferenceNode* node) {
+ DCHECK_NE(node->stage, NodeStage::kPruned);
+ DCHECK(!node->IsPrecolored());
+ node->stage = NodeStage::kPruned;
+ pruned_nodes_.push(node);
+
+ for (InterferenceNode* adj : node->GetAdjacentNodes()) {
+ DCHECK(!adj->GetInterval()->IsSlowPathSafepoint())
+ << "Nodes should never interfere with synthesized safepoint nodes";
+ DCHECK_NE(adj->stage, NodeStage::kPruned) << "Should be no interferences with pruned nodes";
+
+ if (adj->IsPrecolored()) {
+ // No effect on pre-colored nodes; they're never pruned.
+ } else {
+ // Remove the interference.
+ bool was_high_degree = IsHighDegreeNode(adj, num_regs_);
+ DCHECK(adj->ContainsInterference(node))
+ << "Missing reflexive interference from non-fixed node";
+ adj->RemoveInterference(node);
+
+ // Handle transitions from high degree to low degree.
+ if (was_high_degree && IsLowDegreeNode(adj, num_regs_)) {
+ EnableCoalesceOpportunities(adj);
+ for (InterferenceNode* adj_adj : adj->GetAdjacentNodes()) {
+ EnableCoalesceOpportunities(adj_adj);
+ }
+
+ DCHECK_EQ(adj->stage, NodeStage::kSpillWorklist);
+ if (adj->IsMoveRelated()) {
+ adj->stage = NodeStage::kFreezeWorklist;
+ freeze_worklist_.push_back(adj);
+ } else {
+ adj->stage = NodeStage::kSimplifyWorklist;
+ simplify_worklist_.push_back(adj);
+ }
+ }
+ }
+ }
+}
+
+void ColoringIteration::CheckTransitionFromFreezeWorklist(InterferenceNode* node) {
+ if (IsLowDegreeNode(node, num_regs_) && !node->IsMoveRelated()) {
+ DCHECK_EQ(node->stage, NodeStage::kFreezeWorklist);
+ node->stage = NodeStage::kSimplifyWorklist;
+ simplify_worklist_.push_back(node);
+ }
+}
+
+void ColoringIteration::FreezeMoves(InterferenceNode* node) {
+ for (CoalesceOpportunity* opportunity : node->GetCoalesceOpportunities()) {
+ if (opportunity->stage == CoalesceStage::kDefunct) {
+ // Constrained moves should remain constrained, since they will not be considered
+ // during last-chance coalescing.
+ } else {
+ opportunity->stage = CoalesceStage::kInactive;
+ }
+ InterferenceNode* other = opportunity->node_a->GetAlias() == node
+ ? opportunity->node_b->GetAlias()
+ : opportunity->node_a->GetAlias();
+ if (other != node && other->stage == NodeStage::kFreezeWorklist) {
+ DCHECK(IsLowDegreeNode(node, num_regs_));
+ CheckTransitionFromFreezeWorklist(other);
+ }
+ }
+}
+
+bool ColoringIteration::PrecoloredHeuristic(InterferenceNode* from,
+ InterferenceNode* into) {
+ if (!into->IsPrecolored()) {
+ // The uncolored heuristic will cover this case.
+ return false;
+ }
+ if (from->IsPair() || into->IsPair()) {
+ // TODO: Merging from a pair node is currently not supported, since fixed pair nodes
+ // are currently represented as two single fixed nodes in the graph, and `into` is
+ // only one of them. (We may lose the implicit connections to the second one in a merge.)
+ return false;
+ }
+
+ // If all adjacent nodes of `from` are "ok", then we can conservatively merge with `into`.
+ // Reasons an adjacent node `adj` can be "ok":
+ // (1) If `adj` is low degree, interference with `into` will not affect its existing
+ // colorable guarantee. (Notice that coalescing cannot increase its degree.)
+ // (2) If `adj` is pre-colored, it already interferes with `into`. See (3).
+ // (3) If there's already an interference with `into`, coalescing will not add interferences.
+ for (InterferenceNode* adj : from->GetAdjacentNodes()) {
+ if (IsLowDegreeNode(adj, num_regs_) || adj->IsPrecolored() || adj->ContainsInterference(into)) {
+ // Ok.
+ } else {
+ return false;
+ }
+ }
+ return true;
+}
+
+bool ColoringIteration::UncoloredHeuristic(InterferenceNode* from,
+ InterferenceNode* into) {
+ if (into->IsPrecolored()) {
+ // The pre-colored heuristic will handle this case.
+ return false;
+ }
+
+ // Arbitrary cap to improve compile time. Tests show that this has negligible affect
+ // on generated code.
+ if (from->GetOutDegree() + into->GetOutDegree() > 2 * num_regs_) {
+ return false;
+ }
+
+ // It's safe to coalesce two nodes if the resulting node has fewer than `num_regs` neighbors
+ // of high degree. (Low degree neighbors can be ignored, because they will eventually be
+ // pruned from the interference graph in the simplify stage.)
+ size_t high_degree_interferences = 0;
+ for (InterferenceNode* adj : from->GetAdjacentNodes()) {
+ if (IsHighDegreeNode(adj, num_regs_)) {
+ high_degree_interferences += from->EdgeWeightWith(adj);
+ }
+ }
+ for (InterferenceNode* adj : into->GetAdjacentNodes()) {
+ if (IsHighDegreeNode(adj, num_regs_)) {
+ if (from->ContainsInterference(adj)) {
+ // We've already counted this adjacent node.
+ // Furthermore, its degree will decrease if coalescing succeeds. Thus, it's possible that
+ // we should not have counted it at all. (This extends the textbook Briggs coalescing test,
+ // but remains conservative.)
+ if (adj->GetOutDegree() - into->EdgeWeightWith(adj) < num_regs_) {
+ high_degree_interferences -= from->EdgeWeightWith(adj);
+ }
+ } else {
+ high_degree_interferences += into->EdgeWeightWith(adj);
+ }
+ }
+ }
+
+ return high_degree_interferences < num_regs_;
+}
+
+void ColoringIteration::Combine(InterferenceNode* from,
+ InterferenceNode* into) {
+ from->SetAlias(into);
+
+ // Add interferences.
+ for (InterferenceNode* adj : from->GetAdjacentNodes()) {
+ bool was_low_degree = IsLowDegreeNode(adj, num_regs_);
+ AddPotentialInterference(adj, into, /*guaranteed_not_interfering_yet*/ false);
+ if (was_low_degree && IsHighDegreeNode(adj, num_regs_)) {
+ // This is a (temporary) transition to a high degree node. Its degree will decrease again
+ // when we prune `from`, but it's best to be consistent about the current worklist.
+ adj->stage = NodeStage::kSpillWorklist;
+ spill_worklist_.push(adj);
+ }
+ }
+
+ // Add coalesce opportunities.
+ for (CoalesceOpportunity* opportunity : from->GetCoalesceOpportunities()) {
+ if (opportunity->stage != CoalesceStage::kDefunct) {
+ into->AddCoalesceOpportunity(opportunity);
}
}
+ EnableCoalesceOpportunities(from);
+
+ // Prune and update worklists.
+ PruneNode(from);
+ if (IsLowDegreeNode(into, num_regs_)) {
+ // Coalesce(...) takes care of checking for a transition to the simplify worklist.
+ DCHECK_EQ(into->stage, NodeStage::kFreezeWorklist);
+ } else if (into->stage == NodeStage::kFreezeWorklist) {
+ // This is a transition to a high degree node.
+ into->stage = NodeStage::kSpillWorklist;
+ spill_worklist_.push(into);
+ } else {
+ DCHECK(into->stage == NodeStage::kSpillWorklist || into->stage == NodeStage::kPrecolored);
+ }
+}
+
+void ColoringIteration::Coalesce(CoalesceOpportunity* opportunity) {
+ InterferenceNode* from = opportunity->node_a->GetAlias();
+ InterferenceNode* into = opportunity->node_b->GetAlias();
+ DCHECK_NE(from->stage, NodeStage::kPruned);
+ DCHECK_NE(into->stage, NodeStage::kPruned);
+
+ if (from->IsPrecolored()) {
+ // If we have one pre-colored node, make sure it's the `into` node.
+ std::swap(from, into);
+ }
+
+ if (from == into) {
+ // These nodes have already been coalesced.
+ opportunity->stage = CoalesceStage::kDefunct;
+ CheckTransitionFromFreezeWorklist(from);
+ } else if (from->IsPrecolored() || from->ContainsInterference(into)) {
+ // These nodes interfere.
+ opportunity->stage = CoalesceStage::kDefunct;
+ CheckTransitionFromFreezeWorklist(from);
+ CheckTransitionFromFreezeWorklist(into);
+ } else if (PrecoloredHeuristic(from, into)
+ || UncoloredHeuristic(from, into)) {
+ // We can coalesce these nodes.
+ opportunity->stage = CoalesceStage::kDefunct;
+ Combine(from, into);
+ CheckTransitionFromFreezeWorklist(into);
+ } else {
+ // We cannot coalesce, but we may be able to later.
+ opportunity->stage = CoalesceStage::kActive;
+ }
}
// Build a mask with a bit set for each register assigned to some
@@ -888,35 +1783,115 @@ static std::bitset<kMaxNumRegs> BuildConflictMask(Container& intervals) {
return conflict_mask;
}
-bool RegisterAllocatorGraphColor::ColorInterferenceGraph(
- ArenaStdStack<InterferenceNode*>* pruned_nodes,
- size_t num_regs) {
- DCHECK_LE(num_regs, kMaxNumRegs) << "kMaxNumRegs is too small";
+bool RegisterAllocatorGraphColor::IsCallerSave(size_t reg, bool processing_core_regs) {
+ return processing_core_regs
+ ? !codegen_->IsCoreCalleeSaveRegister(reg)
+ : !codegen_->IsCoreCalleeSaveRegister(reg);
+}
+
+static bool RegisterIsAligned(size_t reg) {
+ return reg % 2 == 0;
+}
+
+static size_t FindFirstZeroInConflictMask(std::bitset<kMaxNumRegs> conflict_mask) {
+ // We use CTZ (count trailing zeros) to quickly find the lowest 0 bit.
+ // Note that CTZ is undefined if all bits are 0, so we special-case it.
+ return conflict_mask.all() ? conflict_mask.size() : CTZ(~conflict_mask.to_ulong());
+}
+
+bool ColoringIteration::ColorInterferenceGraph() {
+ DCHECK_LE(num_regs_, kMaxNumRegs) << "kMaxNumRegs is too small";
ArenaVector<LiveInterval*> colored_intervals(
- coloring_attempt_allocator_->Adapter(kArenaAllocRegisterAllocator));
+ allocator_->Adapter(kArenaAllocRegisterAllocator));
bool successful = true;
- while (!pruned_nodes->empty()) {
- InterferenceNode* node = pruned_nodes->top();
- pruned_nodes->pop();
+ while (!pruned_nodes_.empty()) {
+ InterferenceNode* node = pruned_nodes_.top();
+ pruned_nodes_.pop();
LiveInterval* interval = node->GetInterval();
-
- // Search for free register(s).
- // Note that the graph coloring allocator assumes that pair intervals are aligned here,
- // excluding pre-colored pair intervals (which can currently be unaligned on x86).
- std::bitset<kMaxNumRegs> conflict_mask = BuildConflictMask(node->GetAdjacentNodes());
size_t reg = 0;
- if (interval->HasHighInterval()) {
- while (reg < num_regs - 1 && (conflict_mask[reg] || conflict_mask[reg + 1])) {
- reg += 2;
+
+ InterferenceNode* alias = node->GetAlias();
+ if (alias != node) {
+ // This node was coalesced with another.
+ LiveInterval* alias_interval = alias->GetInterval();
+ if (alias_interval->HasRegister()) {
+ reg = alias_interval->GetRegister();
+ DCHECK(!BuildConflictMask(node->GetAdjacentNodes())[reg])
+ << "This node conflicts with the register it was coalesced with";
+ } else {
+ DCHECK(false) << node->GetOutDegree() << " " << alias->GetOutDegree() << " "
+ << "Move coalescing was not conservative, causing a node to be coalesced "
+ << "with another node that could not be colored";
+ if (interval->RequiresRegister()) {
+ successful = false;
+ }
}
} else {
- // We use CTZ (count trailing zeros) to quickly find the lowest available register.
- // Note that CTZ is undefined for 0, so we special-case it.
- reg = conflict_mask.all() ? conflict_mask.size() : CTZ(~conflict_mask.to_ulong());
+ // Search for free register(s).
+ std::bitset<kMaxNumRegs> conflict_mask = BuildConflictMask(node->GetAdjacentNodes());
+ if (interval->HasHighInterval()) {
+ // Note that the graph coloring allocator assumes that pair intervals are aligned here,
+ // excluding pre-colored pair intervals (which can currently be unaligned on x86). If we
+ // change the alignment requirements here, we will have to update the algorithm (e.g.,
+ // be more conservative about the weight of edges adjacent to pair nodes.)
+ while (reg < num_regs_ - 1 && (conflict_mask[reg] || conflict_mask[reg + 1])) {
+ reg += 2;
+ }
+
+ // Try to use a caller-save register first.
+ for (size_t i = 0; i < num_regs_ - 1; i += 2) {
+ bool low_caller_save = register_allocator_->IsCallerSave(i, processing_core_regs_);
+ bool high_caller_save = register_allocator_->IsCallerSave(i + 1, processing_core_regs_);
+ if (!conflict_mask[i] && !conflict_mask[i + 1]) {
+ if (low_caller_save && high_caller_save) {
+ reg = i;
+ break;
+ } else if (low_caller_save || high_caller_save) {
+ reg = i;
+ // Keep looking to try to get both parts in caller-save registers.
+ }
+ }
+ }
+ } else {
+ // Not a pair interval.
+ reg = FindFirstZeroInConflictMask(conflict_mask);
+
+ // Try to use caller-save registers first.
+ for (size_t i = 0; i < num_regs_; ++i) {
+ if (!conflict_mask[i] && register_allocator_->IsCallerSave(i, processing_core_regs_)) {
+ reg = i;
+ break;
+ }
+ }
+ }
+
+ // Last-chance coalescing.
+ for (CoalesceOpportunity* opportunity : node->GetCoalesceOpportunities()) {
+ if (opportunity->stage == CoalesceStage::kDefunct) {
+ continue;
+ }
+ LiveInterval* other_interval = opportunity->node_a->GetAlias() == node
+ ? opportunity->node_b->GetAlias()->GetInterval()
+ : opportunity->node_a->GetAlias()->GetInterval();
+ if (other_interval->HasRegister()) {
+ size_t coalesce_register = other_interval->GetRegister();
+ if (interval->HasHighInterval()) {
+ if (!conflict_mask[coalesce_register] &&
+ !conflict_mask[coalesce_register + 1] &&
+ RegisterIsAligned(coalesce_register)) {
+ reg = coalesce_register;
+ break;
+ }
+ } else if (!conflict_mask[coalesce_register]) {
+ reg = coalesce_register;
+ break;
+ }
+ }
+ }
}
- if (reg < (interval->HasHighInterval() ? num_regs - 1 : num_regs)) {
+ if (reg < (interval->HasHighInterval() ? num_regs_ - 1 : num_regs_)) {
// Assign register.
DCHECK(!interval->HasRegister());
interval->SetRegister(reg);
@@ -930,12 +1905,12 @@ bool RegisterAllocatorGraphColor::ColorInterferenceGraph(
// The interference graph is too dense to color. Make it sparser by
// splitting this live interval.
successful = false;
- SplitAtRegisterUses(interval);
+ register_allocator_->SplitAtRegisterUses(interval);
// We continue coloring, because there may be additional intervals that cannot
// be colored, and that we should split.
} else {
// Spill.
- AllocateSpillSlotFor(interval);
+ register_allocator_->AllocateSpillSlotFor(interval);
}
}
diff --git a/compiler/optimizing/register_allocator_graph_color.h b/compiler/optimizing/register_allocator_graph_color.h
index 0b5af96b40..9dddcea685 100644
--- a/compiler/optimizing/register_allocator_graph_color.h
+++ b/compiler/optimizing/register_allocator_graph_color.h
@@ -34,6 +34,8 @@ class HParallelMove;
class Location;
class SsaLivenessAnalysis;
class InterferenceNode;
+struct CoalesceOpportunity;
+enum class CoalesceKind;
/**
* A graph coloring register allocator.
@@ -60,6 +62,25 @@ class InterferenceNode;
* sparser, so that future coloring attempts may succeed.
* - If the node does not require a register, we simply assign it a location on the stack.
*
+ * If iterative move coalescing is enabled, the algorithm also attempts to conservatively
+ * combine nodes in the graph that would prefer to have the same color. (For example, the output
+ * of a phi instruction would prefer to have the same register as at least one of its inputs.)
+ * There are several additional steps involved with this:
+ * - We look for coalesce opportunities by examining each live interval, a step similar to that
+ * used by linear scan when looking for register hints.
+ * - When pruning the graph, we maintain a worklist of coalesce opportunities, as well as a worklist
+ * of low degree nodes that have associated coalesce opportunities. Only when we run out of
+ * coalesce opportunities do we start pruning coalesce-associated nodes.
+ * - When pruning a node, if any nodes transition from high degree to low degree, we add
+ * associated coalesce opportunities to the worklist, since these opportunities may now succeed.
+ * - Whether two nodes can be combined is decided by two different heuristics--one used when
+ * coalescing uncolored nodes, and one used for coalescing an uncolored node with a colored node.
+ * It is vital that we only combine two nodes if the node that remains is guaranteed to receive
+ * a color. This is because additionally spilling is more costly than failing to coalesce.
+ * - Even if nodes are not coalesced while pruning, we keep the coalesce opportunities around
+ * to be used as last-chance register hints when coloring. If nothing else, we try to use
+ * caller-save registers before callee-save registers.
+ *
* A good reference for graph coloring register allocation is
* "Modern Compiler Implementation in Java" (Andrew W. Appel, 2nd Edition).
*/
@@ -67,7 +88,8 @@ class RegisterAllocatorGraphColor : public RegisterAllocator {
public:
RegisterAllocatorGraphColor(ArenaAllocator* allocator,
CodeGenerator* codegen,
- const SsaLivenessAnalysis& analysis);
+ const SsaLivenessAnalysis& analysis,
+ bool iterative_move_coalescing = true);
~RegisterAllocatorGraphColor() OVERRIDE {}
void AllocateRegisters() OVERRIDE;
@@ -116,26 +138,7 @@ class RegisterAllocatorGraphColor : public RegisterAllocator {
void BlockRegister(Location location, size_t start, size_t end);
void BlockRegisters(size_t start, size_t end, bool caller_save_only = false);
- // Use the intervals collected from instructions to construct an
- // interference graph mapping intervals to adjacency lists.
- // Also, collect synthesized safepoint nodes, used to keep
- // track of live intervals across safepoints.
- void BuildInterferenceGraph(const ArenaVector<LiveInterval*>& intervals,
- ArenaVector<InterferenceNode*>* prunable_nodes,
- ArenaVector<InterferenceNode*>* safepoints);
-
- // Prune nodes from the interference graph to be colored later. Build
- // a stack (pruned_nodes) containing these intervals in an order determined
- // by various heuristics.
- void PruneInterferenceGraph(const ArenaVector<InterferenceNode*>& prunable_nodes,
- size_t num_registers,
- ArenaStdStack<InterferenceNode*>* pruned_nodes);
-
- // Process pruned_intervals to color the interference graph, spilling when
- // necessary. Return true if successful. Else, split some intervals to make
- // the interference graph sparser.
- bool ColorInterferenceGraph(ArenaStdStack<InterferenceNode*>* pruned_nodes,
- size_t num_registers);
+ bool IsCallerSave(size_t reg, bool processing_core_regs);
// Return the maximum number of registers live at safepoints,
// based on the outgoing interference edges of safepoint nodes.
@@ -145,6 +148,10 @@ class RegisterAllocatorGraphColor : public RegisterAllocator {
// and make sure it's ready to be spilled to the stack.
void AllocateSpillSlotFor(LiveInterval* interval);
+ // Whether iterative move coalescing should be performed. Iterative move coalescing
+ // improves code quality, but increases compile time.
+ const bool iterative_move_coalescing_;
+
// Live intervals, split by kind (core and floating point).
// These should not contain high intervals, as those are represented by
// the corresponding low interval throughout register allocation.
@@ -157,10 +164,10 @@ class RegisterAllocatorGraphColor : public RegisterAllocator {
// Safepoints, saved for special handling while processing instructions.
ArenaVector<HInstruction*> safepoints_;
- // Live intervals for specific registers. These become pre-colored nodes
+ // Interference nodes representing specific registers. These are "pre-colored" nodes
// in the interference graph.
- ArenaVector<LiveInterval*> physical_core_intervals_;
- ArenaVector<LiveInterval*> physical_fp_intervals_;
+ ArenaVector<InterferenceNode*> physical_core_nodes_;
+ ArenaVector<InterferenceNode*> physical_fp_nodes_;
// Allocated stack slot counters.
size_t int_spill_slot_counter_;
@@ -184,10 +191,7 @@ class RegisterAllocatorGraphColor : public RegisterAllocator {
size_t max_safepoint_live_core_regs_;
size_t max_safepoint_live_fp_regs_;
- // An arena allocator used for a single graph coloring attempt.
- // Many data structures are cleared between graph coloring attempts, so we reduce
- // total memory usage by using a new arena allocator for each attempt.
- ArenaAllocator* coloring_attempt_allocator_;
+ friend class ColoringIteration;
DISALLOW_COPY_AND_ASSIGN(RegisterAllocatorGraphColor);
};
diff --git a/compiler/optimizing/ssa_liveness_analysis.h b/compiler/optimizing/ssa_liveness_analysis.h
index 346753b775..92788fe6b8 100644
--- a/compiler/optimizing/ssa_liveness_analysis.h
+++ b/compiler/optimizing/ssa_liveness_analysis.h
@@ -514,7 +514,9 @@ class LiveInterval : public ArenaObject<kArenaAllocSsaLiveness> {
// Whether the interval requires a register rather than a stack location.
// If needed for performance, this could be cached.
- bool RequiresRegister() const { return FirstRegisterUse() != kNoLifetime; }
+ bool RequiresRegister() const {
+ return !HasRegister() && FirstRegisterUse() != kNoLifetime;
+ }
size_t FirstUseAfter(size_t position) const {
if (is_temp_) {