|
21 | 21 |
|
22 | 22 | import java.util.HashSet;
|
23 | 23 | import java.util.Set;
|
| 24 | +import java.util.concurrent.atomic.AtomicBoolean; |
| 25 | +import java.util.function.LongSupplier; |
24 | 26 | import java.util.function.Supplier;
|
25 | 27 |
|
26 | 28 | import com.carrotsearch.hppc.ObjectLookupContainer;
|
27 | 29 | import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
|
28 | 30 | import org.apache.logging.log4j.LogManager;
|
29 | 31 | import org.apache.logging.log4j.Logger;
|
| 32 | +import org.elasticsearch.action.ActionListener; |
| 33 | +import org.elasticsearch.action.support.GroupedActionListener; |
30 | 34 | import org.elasticsearch.client.Client;
|
31 | 35 | import org.elasticsearch.cluster.ClusterInfo;
|
32 | 36 | import org.elasticsearch.cluster.ClusterState;
|
@@ -54,11 +58,14 @@ public class DiskThresholdMonitor {
|
54 | 58 | private final Client client;
|
55 | 59 | private final Set<String> nodeHasPassedWatermark = Sets.newConcurrentHashSet();
|
56 | 60 | private final Supplier<ClusterState> clusterStateSupplier;
|
57 |
| - private long lastRunNS; |
| 61 | + private final LongSupplier currentTimeMillisSupplier; |
| 62 | + private long lastRunTimeMillis = Long.MIN_VALUE; |
| 63 | + private final AtomicBoolean checkInProgress = new AtomicBoolean(); |
58 | 64 |
|
59 | 65 | public DiskThresholdMonitor(Settings settings, Supplier<ClusterState> clusterStateSupplier, ClusterSettings clusterSettings,
|
60 |
| - Client client) { |
| 66 | + Client client, LongSupplier currentTimeMillisSupplier) { |
61 | 67 | this.clusterStateSupplier = clusterStateSupplier;
|
| 68 | + this.currentTimeMillisSupplier = currentTimeMillisSupplier; |
62 | 69 | this.diskThresholdSettings = new DiskThresholdSettings(settings, clusterSettings);
|
63 | 70 | this.client = client;
|
64 | 71 | }
|
@@ -92,88 +99,112 @@ private void warnAboutDiskIfNeeded(DiskUsage usage) {
|
92 | 99 | }
|
93 | 100 | }
|
94 | 101 |
|
| 102 | + private void checkFinished() { |
| 103 | + final boolean checkFinished = checkInProgress.compareAndSet(true, false); |
| 104 | + assert checkFinished; |
| 105 | + } |
95 | 106 |
|
96 | 107 | public void onNewInfo(ClusterInfo info) {
|
97 |
| - ImmutableOpenMap<String, DiskUsage> usages = info.getNodeLeastAvailableDiskUsages(); |
98 |
| - if (usages != null) { |
99 |
| - boolean reroute = false; |
100 |
| - String explanation = ""; |
101 |
| - |
102 |
| - // Garbage collect nodes that have been removed from the cluster |
103 |
| - // from the map that tracks watermark crossing |
104 |
| - ObjectLookupContainer<String> nodes = usages.keys(); |
105 |
| - for (String node : nodeHasPassedWatermark) { |
106 |
| - if (nodes.contains(node) == false) { |
107 |
| - nodeHasPassedWatermark.remove(node); |
108 |
| - } |
| 108 | + |
| 109 | + if (checkInProgress.compareAndSet(false, true) == false) { |
| 110 | + logger.info("skipping monitor as a check is already in progress"); |
| 111 | + return; |
| 112 | + } |
| 113 | + |
| 114 | + final ImmutableOpenMap<String, DiskUsage> usages = info.getNodeLeastAvailableDiskUsages(); |
| 115 | + if (usages == null) { |
| 116 | + checkFinished(); |
| 117 | + return; |
| 118 | + } |
| 119 | + |
| 120 | + boolean reroute = false; |
| 121 | + String explanation = ""; |
| 122 | + final long currentTimeMillis = currentTimeMillisSupplier.getAsLong(); |
| 123 | + |
| 124 | + // Garbage collect nodes that have been removed from the cluster |
| 125 | + // from the map that tracks watermark crossing |
| 126 | + final ObjectLookupContainer<String> nodes = usages.keys(); |
| 127 | + for (String node : nodeHasPassedWatermark) { |
| 128 | + if (nodes.contains(node) == false) { |
| 129 | + nodeHasPassedWatermark.remove(node); |
109 | 130 | }
|
110 |
| - ClusterState state = clusterStateSupplier.get(); |
111 |
| - Set<String> indicesToMarkReadOnly = new HashSet<>(); |
112 |
| - for (ObjectObjectCursor<String, DiskUsage> entry : usages) { |
113 |
| - String node = entry.key; |
114 |
| - DiskUsage usage = entry.value; |
115 |
| - warnAboutDiskIfNeeded(usage); |
116 |
| - if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdFloodStage().getBytes() || |
117 |
| - usage.getFreeDiskAsPercentage() < diskThresholdSettings.getFreeDiskThresholdFloodStage()) { |
118 |
| - RoutingNode routingNode = state.getRoutingNodes().node(node); |
119 |
| - if (routingNode != null) { // this might happen if we haven't got the full cluster-state yet?! |
120 |
| - for (ShardRouting routing : routingNode) { |
121 |
| - indicesToMarkReadOnly.add(routing.index().getName()); |
122 |
| - } |
| 131 | + } |
| 132 | + final ClusterState state = clusterStateSupplier.get(); |
| 133 | + final Set<String> indicesToMarkReadOnly = new HashSet<>(); |
| 134 | + |
| 135 | + for (final ObjectObjectCursor<String, DiskUsage> entry : usages) { |
| 136 | + final String node = entry.key; |
| 137 | + final DiskUsage usage = entry.value; |
| 138 | + warnAboutDiskIfNeeded(usage); |
| 139 | + if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdFloodStage().getBytes() || |
| 140 | + usage.getFreeDiskAsPercentage() < diskThresholdSettings.getFreeDiskThresholdFloodStage()) { |
| 141 | + final RoutingNode routingNode = state.getRoutingNodes().node(node); |
| 142 | + if (routingNode != null) { // this might happen if we haven't got the full cluster-state yet?! |
| 143 | + for (ShardRouting routing : routingNode) { |
| 144 | + indicesToMarkReadOnly.add(routing.index().getName()); |
123 | 145 | }
|
124 |
| - } else if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdHigh().getBytes() || |
125 |
| - usage.getFreeDiskAsPercentage() < diskThresholdSettings.getFreeDiskThresholdHigh()) { |
126 |
| - if ((System.nanoTime() - lastRunNS) > diskThresholdSettings.getRerouteInterval().nanos()) { |
127 |
| - lastRunNS = System.nanoTime(); |
| 146 | + } |
| 147 | + } else if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdHigh().getBytes() || |
| 148 | + usage.getFreeDiskAsPercentage() < diskThresholdSettings.getFreeDiskThresholdHigh()) { |
| 149 | + if (lastRunTimeMillis < currentTimeMillis - diskThresholdSettings.getRerouteInterval().millis()) { |
| 150 | + reroute = true; |
| 151 | + explanation = "high disk watermark exceeded on one or more nodes"; |
| 152 | + } else { |
| 153 | + logger.debug("high disk watermark exceeded on {} but an automatic reroute has occurred " + |
| 154 | + "in the last [{}], skipping reroute", |
| 155 | + node, diskThresholdSettings.getRerouteInterval()); |
| 156 | + } |
| 157 | + nodeHasPassedWatermark.add(node); |
| 158 | + } else if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdLow().getBytes() || |
| 159 | + usage.getFreeDiskAsPercentage() < diskThresholdSettings.getFreeDiskThresholdLow()) { |
| 160 | + nodeHasPassedWatermark.add(node); |
| 161 | + } else { |
| 162 | + if (nodeHasPassedWatermark.contains(node)) { |
| 163 | + // The node has previously been over the high or |
| 164 | + // low watermark, but is no longer, so we should |
| 165 | + // reroute so any unassigned shards can be allocated |
| 166 | + // if they are able to be |
| 167 | + if (lastRunTimeMillis < currentTimeMillis - diskThresholdSettings.getRerouteInterval().millis()) { |
128 | 168 | reroute = true;
|
129 |
| - explanation = "high disk watermark exceeded on one or more nodes"; |
| 169 | + explanation = "one or more nodes has gone under the high or low watermark"; |
| 170 | + nodeHasPassedWatermark.remove(node); |
130 | 171 | } else {
|
131 |
| - logger.debug("high disk watermark exceeded on {} but an automatic reroute has occurred " + |
| 172 | + logger.debug("{} has gone below a disk threshold, but an automatic reroute has occurred " + |
132 | 173 | "in the last [{}], skipping reroute",
|
133 | 174 | node, diskThresholdSettings.getRerouteInterval());
|
134 | 175 | }
|
135 |
| - nodeHasPassedWatermark.add(node); |
136 |
| - } else if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdLow().getBytes() || |
137 |
| - usage.getFreeDiskAsPercentage() < diskThresholdSettings.getFreeDiskThresholdLow()) { |
138 |
| - nodeHasPassedWatermark.add(node); |
139 |
| - } else { |
140 |
| - if (nodeHasPassedWatermark.contains(node)) { |
141 |
| - // The node has previously been over the high or |
142 |
| - // low watermark, but is no longer, so we should |
143 |
| - // reroute so any unassigned shards can be allocated |
144 |
| - // if they are able to be |
145 |
| - if ((System.nanoTime() - lastRunNS) > diskThresholdSettings.getRerouteInterval().nanos()) { |
146 |
| - lastRunNS = System.nanoTime(); |
147 |
| - reroute = true; |
148 |
| - explanation = "one or more nodes has gone under the high or low watermark"; |
149 |
| - nodeHasPassedWatermark.remove(node); |
150 |
| - } else { |
151 |
| - logger.debug("{} has gone below a disk threshold, but an automatic reroute has occurred " + |
152 |
| - "in the last [{}], skipping reroute", |
153 |
| - node, diskThresholdSettings.getRerouteInterval()); |
154 |
| - } |
155 |
| - } |
156 | 176 | }
|
157 | 177 | }
|
158 |
| - if (reroute) { |
159 |
| - logger.info("rerouting shards: [{}]", explanation); |
160 |
| - reroute(); |
161 |
| - } |
162 |
| - indicesToMarkReadOnly.removeIf(index -> state.getBlocks().indexBlocked(ClusterBlockLevel.WRITE, index)); |
163 |
| - if (indicesToMarkReadOnly.isEmpty() == false) { |
164 |
| - markIndicesReadOnly(indicesToMarkReadOnly); |
165 |
| - } |
| 178 | + } |
| 179 | + |
| 180 | + final ActionListener<Void> listener = new GroupedActionListener<>(ActionListener.wrap(this::checkFinished), 2); |
| 181 | + |
| 182 | + if (reroute) { |
| 183 | + logger.info("rerouting shards: [{}]", explanation); |
| 184 | + reroute(ActionListener.wrap(r -> { |
| 185 | + lastRunTimeMillis = currentTimeMillisSupplier.getAsLong(); |
| 186 | + listener.onResponse(r); |
| 187 | + }, listener::onFailure)); |
| 188 | + } else { |
| 189 | + listener.onResponse(null); |
| 190 | + } |
| 191 | + indicesToMarkReadOnly.removeIf(index -> state.getBlocks().indexBlocked(ClusterBlockLevel.WRITE, index)); |
| 192 | + if (indicesToMarkReadOnly.isEmpty() == false) { |
| 193 | + markIndicesReadOnly(indicesToMarkReadOnly, listener); |
| 194 | + } else { |
| 195 | + listener.onResponse(null); |
166 | 196 | }
|
167 | 197 | }
|
168 | 198 |
|
169 |
| - protected void markIndicesReadOnly(Set<String> indicesToMarkReadOnly) { |
| 199 | + protected void markIndicesReadOnly(Set<String> indicesToMarkReadOnly, ActionListener<Void> listener) { |
170 | 200 | // set read-only block but don't block on the response
|
171 |
| - client.admin().indices().prepareUpdateSettings(indicesToMarkReadOnly.toArray(Strings.EMPTY_ARRAY)). |
172 |
| - setSettings(Settings.builder().put(IndexMetaData.SETTING_READ_ONLY_ALLOW_DELETE, true).build()).execute(); |
| 201 | + client.admin().indices().prepareUpdateSettings(indicesToMarkReadOnly.toArray(Strings.EMPTY_ARRAY)) |
| 202 | + .setSettings(Settings.builder().put(IndexMetaData.SETTING_READ_ONLY_ALLOW_DELETE, true).build()) |
| 203 | + .execute(ActionListener.map(listener, r -> null)); |
173 | 204 | }
|
174 | 205 |
|
175 |
| - protected void reroute() { |
| 206 | + protected void reroute(ActionListener<Void> listener) { |
176 | 207 | // Execute an empty reroute, but don't block on the response
|
177 |
| - client.admin().cluster().prepareReroute().execute(); |
| 208 | + client.admin().cluster().prepareReroute().execute(ActionListener.map(listener, r -> null)); |
178 | 209 | }
|
179 | 210 | }
|
0 commit comments