|
9 | 9 |
|
10 | 10 | package org.elasticsearch.cluster.routing.allocation.allocator; |
11 | 11 |
|
| 12 | +import java.util.HashMap; |
| 13 | +import java.util.List; |
| 14 | +import java.util.Map; |
| 15 | + |
12 | 16 | /** |
13 | 17 | * Summarizes the impact to the cluster as a result of a rebalancing round. |
14 | 18 | * |
15 | | - * @param numberOfShardsToMove The number of shard moves required to move from the previous desired balance to the new one. |
| 19 | + * @param nodeNameToWeightChanges The shard balance weight changes for each node (by name), comparing a previous DesiredBalance shard |
| 20 | + * allocation to a new DesiredBalance allocation. |
| 21 | + * @param numberOfShardsToMove The number of shard moves required to move from the previous desired balance to the new one. Does not include |
| 22 | + * new (index creation) or removed (index deletion) shard assignements. |
16 | 23 | */ |
17 | | -public record BalancingRoundSummary(long numberOfShardsToMove) { |
| 24 | +public record BalancingRoundSummary(Map<String, NodesWeightsChanges> nodeNameToWeightChanges, long numberOfShardsToMove) { |
| 25 | + |
| 26 | + /** |
| 27 | + * Represents the change in weights for a node going from an old DesiredBalance to a new DesiredBalance |
| 28 | + * Saves the node weights of an old DesiredBalance, along with a diff against a newer DesiredBalance. |
| 29 | + * |
| 30 | + * @param baseWeights The starting {@link DesiredBalanceMetrics.NodeWeightStats} of a previous DesiredBalance. |
| 31 | + * @param weightsDiff The difference between the {@code baseWeights} and a new DesiredBalance. |
| 32 | + */ |
| 33 | + record NodesWeightsChanges(DesiredBalanceMetrics.NodeWeightStats baseWeights, NodeWeightsDiff weightsDiff) {} |
| 34 | + |
| 35 | + /** |
| 36 | + * Represents the change of shard balance weights for a node, comparing an older DesiredBalance with the latest DesiredBalance. |
| 37 | + * |
| 38 | + * @param shardCountDiff How many more, or less, shards are assigned to the node in the latest DesiredBalance. |
| 39 | + * @param diskUsageInBytesDiff How much more, or less, disk is used by shards assigned to the node in the latest DesiredBalance. |
| 40 | + * @param writeLoadDiff How much more, or less, write load is estimated for shards assigned to the node in the latest DesiredBalance. |
| 41 | + * @param totalWeightDiff How much more, or less, the total weight is of shards assigned to the node in the latest DesiredBalance. |
| 42 | + */ |
| 43 | + record NodeWeightsDiff(long shardCountDiff, double diskUsageInBytesDiff, double writeLoadDiff, double totalWeightDiff) { |
| 44 | + |
| 45 | + /** |
| 46 | + * Creates a diff where the {@code base} weights will be subtracted from the {@code next} weights, to show the changes made to reach |
| 47 | + * the {@code next} weights. |
| 48 | + * |
| 49 | + * @param base has the original weights |
| 50 | + * @param next has the new weights |
| 51 | + * @return The diff of ({@code next} - {@code base}) |
| 52 | + */ |
| 53 | + public static NodeWeightsDiff create(DesiredBalanceMetrics.NodeWeightStats base, DesiredBalanceMetrics.NodeWeightStats next) { |
| 54 | + return new NodeWeightsDiff( |
| 55 | + next.shardCount() - base.shardCount(), |
| 56 | + next.diskUsageInBytes() - base.diskUsageInBytes(), |
| 57 | + next.writeLoad() - base.writeLoad(), |
| 58 | + next.nodeWeight() - base.nodeWeight() |
| 59 | + ); |
| 60 | + } |
| 61 | + |
| 62 | + /** |
| 63 | + * Creates a new {@link NodeWeightsDiff} summing this instance's values with {@code otherDiff}'s values. |
| 64 | + */ |
| 65 | + public NodeWeightsDiff combine(NodeWeightsDiff otherDiff) { |
| 66 | + return new NodeWeightsDiff( |
| 67 | + this.shardCountDiff + otherDiff.shardCountDiff, |
| 68 | + this.diskUsageInBytesDiff + otherDiff.diskUsageInBytesDiff, |
| 69 | + this.writeLoadDiff + otherDiff.writeLoadDiff, |
| 70 | + this.totalWeightDiff + otherDiff.totalWeightDiff |
| 71 | + ); |
| 72 | + } |
| 73 | + } |
18 | 74 |
|
19 | 75 | @Override |
20 | 76 | public String toString() { |
21 | | - return "BalancingRoundSummary{" + "numberOfShardsToMove=" + numberOfShardsToMove + '}'; |
| 77 | + return "BalancingRoundSummary{" |
| 78 | + + "nodeNameToWeightChanges" |
| 79 | + + nodeNameToWeightChanges |
| 80 | + + ", numberOfShardsToMove=" |
| 81 | + + numberOfShardsToMove |
| 82 | + + '}'; |
| 83 | + } |
| 84 | + |
| 85 | + /** |
| 86 | + * Holds combined {@link BalancingRoundSummary} results. Essentially holds a list of the balancing events and the summed up changes |
| 87 | + * across all those events: what allocation work was done across some period of time. |
| 88 | + * TODO: WIP ES-10341 |
| 89 | + * |
| 90 | + * Note that each balancing round summary is the difference between, at the time, latest desired balance and the previous desired |
| 91 | + * balance. Each summary represents a step towards the next desired balance, which is based on presuming the previous desired balance is |
| 92 | + * reached. So combining them is roughly the difference between the first summary's previous desired balance and the last summary's |
| 93 | + * latest desired balance. |
| 94 | + * |
| 95 | + * @param numberOfBalancingRounds How many balancing round summaries are combined in this report. |
| 96 | + * @param nodeNameToWeightChanges |
| 97 | + * @param numberOfShardMoves The sum of shard moves for each balancing round being combined into a single summary. |
| 98 | + */ |
| 99 | + public record CombinedBalancingRoundSummary( |
| 100 | + int numberOfBalancingRounds, |
| 101 | + Map<String, NodesWeightsChanges> nodeNameToWeightChanges, |
| 102 | + long numberOfShardMoves |
| 103 | + ) { |
| 104 | + |
| 105 | + public static final CombinedBalancingRoundSummary EMPTY_RESULTS = new CombinedBalancingRoundSummary(0, new HashMap<>(), 0); |
| 106 | + |
| 107 | + /** |
| 108 | + * Merges multiple {@link BalancingRoundSummary} summaries into a single {@link CombinedBalancingRoundSummary}. |
| 109 | + */ |
| 110 | + public static CombinedBalancingRoundSummary combine(List<BalancingRoundSummary> summaries) { |
| 111 | + if (summaries.isEmpty()) { |
| 112 | + return EMPTY_RESULTS; |
| 113 | + } |
| 114 | + |
| 115 | + // We will loop through the summaries and sum the weight diffs for each node entry. |
| 116 | + Map<String, NodesWeightsChanges> combinedNodeNameToWeightChanges = new HashMap<>(); |
| 117 | + |
| 118 | + // Number of shards moves are simply summed across summaries. Each new balancing round is built upon the last one, so it is |
| 119 | + // possible that a shard is reassigned back to a node before it even moves away, and that will still be counted as 2 moves here. |
| 120 | + long numberOfShardMoves = 0; |
| 121 | + |
| 122 | + // Total number of summaries that are being combined. |
| 123 | + int numSummaries = 0; |
| 124 | + |
| 125 | + var iterator = summaries.iterator(); |
| 126 | + while (iterator.hasNext()) { |
| 127 | + var summary = iterator.next(); |
| 128 | + |
| 129 | + // We'll build the weight changes by keeping the node weight base from the first summary in which a node appears and then |
| 130 | + // summing the weight diffs in each summary to get total weight diffs across summaries. |
| 131 | + for (var nodeNameAndWeights : summary.nodeNameToWeightChanges.entrySet()) { |
| 132 | + var combined = combinedNodeNameToWeightChanges.get(nodeNameAndWeights.getKey()); |
| 133 | + if (combined == null) { |
| 134 | + // Either this is the first summary, and combinedNodeNameToWeightChanges hasn't been initialized yet for this node; |
| 135 | + // or a later balancing round had a new node. Either way, initialize the node entry with the weight changes from the |
| 136 | + // first summary in which it appears. |
| 137 | + combinedNodeNameToWeightChanges.put(nodeNameAndWeights.getKey(), nodeNameAndWeights.getValue()); |
| 138 | + } else { |
| 139 | + // We have at least two summaries containing this node, so let's combine them. |
| 140 | + var newCombinedChanges = new NodesWeightsChanges( |
| 141 | + combined.baseWeights, |
| 142 | + combined.weightsDiff.combine(nodeNameAndWeights.getValue().weightsDiff()) |
| 143 | + ); |
| 144 | + combinedNodeNameToWeightChanges.put(nodeNameAndWeights.getKey(), newCombinedChanges); |
| 145 | + } |
| 146 | + } |
| 147 | + |
| 148 | + ++numSummaries; |
| 149 | + numberOfShardMoves += summary.numberOfShardsToMove(); |
| 150 | + } |
| 151 | + |
| 152 | + return new CombinedBalancingRoundSummary(numSummaries, combinedNodeNameToWeightChanges, numberOfShardMoves); |
| 153 | + } |
| 154 | + |
22 | 155 | } |
23 | 156 |
|
24 | 157 | } |
0 commit comments