-
Notifications
You must be signed in to change notification settings - Fork 1.9k
/
AsyncShardFetch.java
478 lines (436 loc) · 19.4 KB
/
AsyncShardFetch.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.gateway;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.opensearch.OpenSearchTimeoutException;
import org.opensearch.ExceptionsHelper;
import org.opensearch.action.ActionListener;
import org.opensearch.action.FailedNodeException;
import org.opensearch.action.support.nodes.BaseNodeResponse;
import org.opensearch.action.support.nodes.BaseNodesResponse;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.cluster.node.DiscoveryNodes;
import org.opensearch.cluster.routing.allocation.RoutingAllocation;
import org.opensearch.common.Nullable;
import org.opensearch.common.lease.Releasable;
import org.opensearch.common.util.concurrent.OpenSearchRejectedExecutionException;
import org.opensearch.index.shard.ShardId;
import org.opensearch.transport.ReceiveTimeoutTransportException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import static java.util.Collections.emptySet;
import static java.util.Collections.unmodifiableSet;
/**
* Allows to asynchronously fetch shard related data from other nodes for allocation, without blocking
* the cluster update thread.
* <p>
* The async fetch logic maintains a map of which nodes are being fetched from in an async manner,
* and once the results are back, it makes sure to schedule a reroute to make sure those results will
* be taken into account.
*
* @opensearch.internal
*/
public abstract class AsyncShardFetch<T extends BaseNodeResponse> implements Releasable {
/**
* An action that lists the relevant shard data that needs to be fetched.
*/
public interface Lister<NodesResponse extends BaseNodesResponse<NodeResponse>, NodeResponse extends BaseNodeResponse> {
void list(ShardId shardId, @Nullable String customDataPath, DiscoveryNode[] nodes, ActionListener<NodesResponse> listener);
}
protected final Logger logger;
protected final String type;
protected final ShardId shardId;
protected final String customDataPath;
private final Lister<BaseNodesResponse<T>, T> action;
private final Map<String, NodeEntry<T>> cache = new HashMap<>();
private final Set<String> nodesToIgnore = new HashSet<>();
private final AtomicLong round = new AtomicLong();
private boolean closed;
@SuppressWarnings("unchecked")
protected AsyncShardFetch(
Logger logger,
String type,
ShardId shardId,
String customDataPath,
Lister<? extends BaseNodesResponse<T>, T> action
) {
this.logger = logger;
this.type = type;
this.shardId = Objects.requireNonNull(shardId);
this.customDataPath = Objects.requireNonNull(customDataPath);
this.action = (Lister<BaseNodesResponse<T>, T>) action;
}
@Override
public synchronized void close() {
this.closed = true;
}
/**
* Returns the number of async fetches that are currently ongoing.
*/
public synchronized int getNumberOfInFlightFetches() {
int count = 0;
for (NodeEntry<T> nodeEntry : cache.values()) {
if (nodeEntry.isFetching()) {
count++;
}
}
return count;
}
/**
* Fetches the data for the relevant shard. If there any ongoing async fetches going on, or new ones have
* been initiated by this call, the result will have no data.
* <p>
* The ignoreNodes are nodes that are supposed to be ignored for this round, since fetching is async, we need
* to keep them around and make sure we add them back when all the responses are fetched and returned.
*/
public synchronized FetchResult<T> fetchData(DiscoveryNodes nodes, Set<String> ignoreNodes) {
if (closed) {
throw new IllegalStateException(shardId + ": can't fetch data on closed async fetch");
}
nodesToIgnore.addAll(ignoreNodes);
fillShardCacheWithDataNodes(cache, nodes);
List<NodeEntry<T>> nodesToFetch = findNodesToFetch(cache);
if (nodesToFetch.isEmpty() == false) {
// mark all node as fetching and go ahead and async fetch them
// use a unique round id to detect stale responses in processAsyncFetch
final long fetchingRound = round.incrementAndGet();
for (NodeEntry<T> nodeEntry : nodesToFetch) {
nodeEntry.markAsFetching(fetchingRound);
}
DiscoveryNode[] discoNodesToFetch = nodesToFetch.stream()
.map(NodeEntry::getNodeId)
.map(nodes::get)
.toArray(DiscoveryNode[]::new);
asyncFetch(discoNodesToFetch, fetchingRound);
}
// if we are still fetching, return null to indicate it
if (hasAnyNodeFetching(cache)) {
return new FetchResult<>(shardId, null, emptySet());
} else {
// nothing to fetch, yay, build the return value
Map<DiscoveryNode, T> fetchData = new HashMap<>();
Set<String> failedNodes = new HashSet<>();
for (Iterator<Map.Entry<String, NodeEntry<T>>> it = cache.entrySet().iterator(); it.hasNext();) {
Map.Entry<String, NodeEntry<T>> entry = it.next();
String nodeId = entry.getKey();
NodeEntry<T> nodeEntry = entry.getValue();
DiscoveryNode node = nodes.get(nodeId);
if (node != null) {
if (nodeEntry.isFailed()) {
// if its failed, remove it from the list of nodes, so if this run doesn't work
// we try again next round to fetch it again
it.remove();
failedNodes.add(nodeEntry.getNodeId());
} else {
if (nodeEntry.getValue() != null) {
fetchData.put(node, nodeEntry.getValue());
}
}
}
}
Set<String> allIgnoreNodes = unmodifiableSet(new HashSet<>(nodesToIgnore));
// clear the nodes to ignore, we had a successful run in fetching everything we can
// we need to try them if another full run is needed
nodesToIgnore.clear();
// if at least one node failed, make sure to have a protective reroute
// here, just case this round won't find anything, and we need to retry fetching data
if (failedNodes.isEmpty() == false || allIgnoreNodes.isEmpty() == false) {
reroute(shardId, "nodes failed [" + failedNodes.size() + "], ignored [" + allIgnoreNodes.size() + "]");
}
return new FetchResult<>(shardId, fetchData, allIgnoreNodes);
}
}
/**
* Called by the response handler of the async action to fetch data. Verifies that its still working
* on the same cache generation, otherwise the results are discarded. It then goes and fills the relevant data for
* the shard (response + failures), issuing a reroute at the end of it to make sure there will be another round
* of allocations taking this new data into account.
*/
protected synchronized void processAsyncFetch(List<T> responses, List<FailedNodeException> failures, long fetchingRound) {
if (closed) {
// we are closed, no need to process this async fetch at all
logger.trace("{} ignoring fetched [{}] results, already closed", shardId, type);
return;
}
logger.trace("{} processing fetched [{}] results", shardId, type);
if (responses != null) {
for (T response : responses) {
NodeEntry<T> nodeEntry = cache.get(response.getNode().getId());
if (nodeEntry != null) {
if (nodeEntry.getFetchingRound() != fetchingRound) {
assert nodeEntry.getFetchingRound() > fetchingRound : "node entries only replaced by newer rounds";
logger.trace(
"{} received response for [{}] from node {} for an older fetching round (expected: {} but was: {})",
shardId,
nodeEntry.getNodeId(),
type,
nodeEntry.getFetchingRound(),
fetchingRound
);
} else if (nodeEntry.isFailed()) {
logger.trace(
"{} node {} has failed for [{}] (failure [{}])",
shardId,
nodeEntry.getNodeId(),
type,
nodeEntry.getFailure()
);
} else {
// if the entry is there, for the right fetching round and not marked as failed already, process it
logger.trace("{} marking {} as done for [{}], result is [{}]", shardId, nodeEntry.getNodeId(), type, response);
nodeEntry.doneFetching(response);
}
}
}
}
if (failures != null) {
for (FailedNodeException failure : failures) {
logger.trace("{} processing failure {} for [{}]", shardId, failure, type);
NodeEntry<T> nodeEntry = cache.get(failure.nodeId());
if (nodeEntry != null) {
if (nodeEntry.getFetchingRound() != fetchingRound) {
assert nodeEntry.getFetchingRound() > fetchingRound : "node entries only replaced by newer rounds";
logger.trace(
"{} received failure for [{}] from node {} for an older fetching round (expected: {} but was: {})",
shardId,
nodeEntry.getNodeId(),
type,
nodeEntry.getFetchingRound(),
fetchingRound
);
} else if (nodeEntry.isFailed() == false) {
// if the entry is there, for the right fetching round and not marked as failed already, process it
Throwable unwrappedCause = ExceptionsHelper.unwrapCause(failure.getCause());
// if the request got rejected or timed out, we need to try it again next time...
if (unwrappedCause instanceof OpenSearchRejectedExecutionException
|| unwrappedCause instanceof ReceiveTimeoutTransportException
|| unwrappedCause instanceof OpenSearchTimeoutException) {
nodeEntry.restartFetching();
} else {
logger.warn(
() -> new ParameterizedMessage(
"{}: failed to list shard for {} on node [{}]",
shardId,
type,
failure.nodeId()
),
failure
);
nodeEntry.doneFetching(failure.getCause());
}
}
}
}
}
reroute(shardId, "post_response");
}
/**
* Implement this in order to scheduled another round that causes a call to fetch data.
*/
protected abstract void reroute(ShardId shardId, String reason);
/**
* Clear cache for node, ensuring next fetch will fetch a fresh copy.
*/
synchronized void clearCacheForNode(String nodeId) {
cache.remove(nodeId);
}
/**
* Fills the shard fetched data with new (data) nodes and a fresh NodeEntry, and removes from
* it nodes that are no longer part of the state.
*/
private void fillShardCacheWithDataNodes(Map<String, NodeEntry<T>> shardCache, DiscoveryNodes nodes) {
// verify that all current data nodes are there
for (ObjectObjectCursor<String, DiscoveryNode> cursor : nodes.getDataNodes()) {
DiscoveryNode node = cursor.value;
if (shardCache.containsKey(node.getId()) == false) {
shardCache.put(node.getId(), new NodeEntry<T>(node.getId()));
}
}
// remove nodes that are not longer part of the data nodes set
shardCache.keySet().removeIf(nodeId -> !nodes.nodeExists(nodeId));
}
/**
* Finds all the nodes that need to be fetched. Those are nodes that have no
* data, and are not in fetch mode.
*/
private List<NodeEntry<T>> findNodesToFetch(Map<String, NodeEntry<T>> shardCache) {
List<NodeEntry<T>> nodesToFetch = new ArrayList<>();
for (NodeEntry<T> nodeEntry : shardCache.values()) {
if (nodeEntry.hasData() == false && nodeEntry.isFetching() == false) {
nodesToFetch.add(nodeEntry);
}
}
return nodesToFetch;
}
/**
* Are there any nodes that are fetching data?
*/
private boolean hasAnyNodeFetching(Map<String, NodeEntry<T>> shardCache) {
for (NodeEntry<T> nodeEntry : shardCache.values()) {
if (nodeEntry.isFetching()) {
return true;
}
}
return false;
}
/**
* Async fetches data for the provided shard with the set of nodes that need to be fetched from.
*/
// visible for testing
void asyncFetch(final DiscoveryNode[] nodes, long fetchingRound) {
logger.trace("{} fetching [{}] from {}", shardId, type, nodes);
action.list(shardId, customDataPath, nodes, new ActionListener<BaseNodesResponse<T>>() {
@Override
public void onResponse(BaseNodesResponse<T> response) {
processAsyncFetch(response.getNodes(), response.failures(), fetchingRound);
}
@Override
public void onFailure(Exception e) {
List<FailedNodeException> failures = new ArrayList<>(nodes.length);
for (final DiscoveryNode node : nodes) {
failures.add(new FailedNodeException(node.getId(), "total failure in fetching", e));
}
processAsyncFetch(null, failures, fetchingRound);
}
});
}
/**
* The result of a fetch operation. Make sure to first check {@link #hasData()} before
* fetching the actual data.
*/
public static class FetchResult<T extends BaseNodeResponse> {
private final ShardId shardId;
private final Map<DiscoveryNode, T> data;
private final Set<String> ignoreNodes;
public FetchResult(ShardId shardId, Map<DiscoveryNode, T> data, Set<String> ignoreNodes) {
this.shardId = shardId;
this.data = data;
this.ignoreNodes = ignoreNodes;
}
/**
* Does the result actually contain data? If not, then there are on going fetch
* operations happening, and it should wait for it.
*/
public boolean hasData() {
return data != null;
}
/**
* Returns the actual data, note, make sure to check {@link #hasData()} first and
* only use this when there is an actual data.
*/
public Map<DiscoveryNode, T> getData() {
assert data != null : "getData should only be called if there is data to be fetched, please check hasData first";
return this.data;
}
/**
* Process any changes needed to the allocation based on this fetch result.
*/
public void processAllocation(RoutingAllocation allocation) {
for (String ignoreNode : ignoreNodes) {
allocation.addIgnoreShardForNode(shardId, ignoreNode);
}
}
}
/**
* A node entry, holding the state of the fetched data for a specific shard
* for a giving node.
*/
static class NodeEntry<T> {
private final String nodeId;
private boolean fetching;
@Nullable
private T value;
private boolean valueSet;
private Throwable failure;
private long fetchingRound;
NodeEntry(String nodeId) {
this.nodeId = nodeId;
}
String getNodeId() {
return this.nodeId;
}
boolean isFetching() {
return fetching;
}
void markAsFetching(long fetchingRound) {
assert fetching == false : "double marking a node as fetching";
this.fetching = true;
this.fetchingRound = fetchingRound;
}
void doneFetching(T value) {
assert fetching : "setting value but not in fetching mode";
assert failure == null : "setting value when failure already set";
this.valueSet = true;
this.value = value;
this.fetching = false;
}
void doneFetching(Throwable failure) {
assert fetching : "setting value but not in fetching mode";
assert valueSet == false : "setting failure when already set value";
assert failure != null : "setting failure can't be null";
this.failure = failure;
this.fetching = false;
}
void restartFetching() {
assert fetching : "restarting fetching, but not in fetching mode";
assert valueSet == false : "value can't be set when restarting fetching";
assert failure == null : "failure can't be set when restarting fetching";
this.fetching = false;
}
boolean isFailed() {
return failure != null;
}
boolean hasData() {
return valueSet || failure != null;
}
Throwable getFailure() {
assert hasData() : "getting failure when data has not been fetched";
return failure;
}
@Nullable
T getValue() {
assert failure == null : "trying to fetch value, but its marked as failed, check isFailed";
assert valueSet : "value is not set, hasn't been fetched yet";
return value;
}
long getFetchingRound() {
return fetchingRound;
}
}
}