|
@@ -16,6 +16,7 @@ import org.elasticsearch.action.support.ActionFilters;
|
|
|
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
|
|
|
import org.elasticsearch.action.support.HandledTransportAction;
|
|
|
import org.elasticsearch.action.support.IndicesOptions;
|
|
|
+import org.elasticsearch.action.support.NodeResponseTracker;
|
|
|
import org.elasticsearch.action.support.TransportActions;
|
|
|
import org.elasticsearch.action.support.broadcast.BroadcastRequest;
|
|
|
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
|
|
@@ -51,7 +52,6 @@ import java.util.HashMap;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
-import java.util.concurrent.atomic.AtomicReferenceArray;
|
|
|
import java.util.function.Consumer;
|
|
|
|
|
|
/**
|
|
@@ -118,28 +118,29 @@ public abstract class TransportBroadcastByNodeAction<
|
|
|
|
|
|
private Response newResponse(
|
|
|
Request request,
|
|
|
- AtomicReferenceArray<?> responses,
|
|
|
+ NodeResponseTracker nodeResponseTracker,
|
|
|
int unavailableShardCount,
|
|
|
Map<String, List<ShardRouting>> nodes,
|
|
|
ClusterState clusterState
|
|
|
- ) {
|
|
|
+ ) throws NodeResponseTracker.DiscardedResponsesException {
|
|
|
int totalShards = 0;
|
|
|
int successfulShards = 0;
|
|
|
List<ShardOperationResult> broadcastByNodeResponses = new ArrayList<>();
|
|
|
List<DefaultShardOperationFailedException> exceptions = new ArrayList<>();
|
|
|
- for (int i = 0; i < responses.length(); i++) {
|
|
|
- if (responses.get(i)instanceof FailedNodeException exception) {
|
|
|
+ for (int i = 0; i < nodeResponseTracker.getExpectedResponseCount(); i++) {
|
|
|
+ Object response = nodeResponseTracker.getResponse(i);
|
|
|
+ if (response instanceof FailedNodeException exception) {
|
|
|
totalShards += nodes.get(exception.nodeId()).size();
|
|
|
for (ShardRouting shard : nodes.get(exception.nodeId())) {
|
|
|
exceptions.add(new DefaultShardOperationFailedException(shard.getIndexName(), shard.getId(), exception));
|
|
|
}
|
|
|
} else {
|
|
|
@SuppressWarnings("unchecked")
|
|
|
- NodeResponse response = (NodeResponse) responses.get(i);
|
|
|
- broadcastByNodeResponses.addAll(response.results);
|
|
|
- totalShards += response.getTotalShards();
|
|
|
- successfulShards += response.getSuccessfulShards();
|
|
|
- for (BroadcastShardOperationFailedException throwable : response.getExceptions()) {
|
|
|
+ NodeResponse nodeResponse = (NodeResponse) response;
|
|
|
+ broadcastByNodeResponses.addAll(nodeResponse.results);
|
|
|
+ totalShards += nodeResponse.getTotalShards();
|
|
|
+ successfulShards += nodeResponse.getSuccessfulShards();
|
|
|
+ for (BroadcastShardOperationFailedException throwable : nodeResponse.getExceptions()) {
|
|
|
if (TransportActions.isShardNotAvailableException(throwable) == false) {
|
|
|
exceptions.add(
|
|
|
new DefaultShardOperationFailedException(
|
|
@@ -256,16 +257,15 @@ public abstract class TransportBroadcastByNodeAction<
|
|
|
new AsyncAction(task, request, listener).start();
|
|
|
}
|
|
|
|
|
|
- protected class AsyncAction {
|
|
|
+ protected class AsyncAction implements CancellableTask.CancellationListener {
|
|
|
private final Task task;
|
|
|
private final Request request;
|
|
|
private final ActionListener<Response> listener;
|
|
|
private final ClusterState clusterState;
|
|
|
private final DiscoveryNodes nodes;
|
|
|
private final Map<String, List<ShardRouting>> nodeIds;
|
|
|
- private final AtomicReferenceArray<Object> responses;
|
|
|
- private final AtomicInteger counter = new AtomicInteger();
|
|
|
private final int unavailableShardCount;
|
|
|
+ private final NodeResponseTracker nodeResponseTracker;
|
|
|
|
|
|
protected AsyncAction(Task task, Request request, ActionListener<Response> listener) {
|
|
|
this.task = task;
|
|
@@ -312,10 +312,13 @@ public abstract class TransportBroadcastByNodeAction<
|
|
|
|
|
|
}
|
|
|
this.unavailableShardCount = unavailableShardCount;
|
|
|
- responses = new AtomicReferenceArray<>(nodeIds.size());
|
|
|
+ nodeResponseTracker = new NodeResponseTracker(nodeIds.size());
|
|
|
}
|
|
|
|
|
|
public void start() {
|
|
|
+ if (task instanceof CancellableTask cancellableTask) {
|
|
|
+ cancellableTask.addListener(this);
|
|
|
+ }
|
|
|
if (nodeIds.size() == 0) {
|
|
|
try {
|
|
|
onCompletion();
|
|
@@ -373,38 +376,34 @@ public abstract class TransportBroadcastByNodeAction<
|
|
|
logger.trace("received response for [{}] from node [{}]", actionName, node.getId());
|
|
|
}
|
|
|
|
|
|
- // this is defensive to protect against the possibility of double invocation
|
|
|
- // the current implementation of TransportService#sendRequest guards against this
|
|
|
- // but concurrency is hard, safety is important, and the small performance loss here does not matter
|
|
|
- if (responses.compareAndSet(nodeIndex, null, response)) {
|
|
|
- if (counter.incrementAndGet() == responses.length()) {
|
|
|
- onCompletion();
|
|
|
- }
|
|
|
+ if (nodeResponseTracker.trackResponseAndCheckIfLast(nodeIndex, response)) {
|
|
|
+ onCompletion();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
protected void onNodeFailure(DiscoveryNode node, int nodeIndex, Throwable t) {
|
|
|
String nodeId = node.getId();
|
|
|
logger.debug(new ParameterizedMessage("failed to execute [{}] on node [{}]", actionName, nodeId), t);
|
|
|
-
|
|
|
- // this is defensive to protect against the possibility of double invocation
|
|
|
- // the current implementation of TransportService#sendRequest guards against this
|
|
|
- // but concurrency is hard, safety is important, and the small performance loss here does not matter
|
|
|
- if (responses.compareAndSet(nodeIndex, null, new FailedNodeException(nodeId, "Failed node [" + nodeId + "]", t))) {
|
|
|
- if (counter.incrementAndGet() == responses.length()) {
|
|
|
- onCompletion();
|
|
|
- }
|
|
|
+ if (nodeResponseTracker.trackResponseAndCheckIfLast(
|
|
|
+ nodeIndex,
|
|
|
+ new FailedNodeException(nodeId, "Failed node [" + nodeId + "]", t)
|
|
|
+ )) {
|
|
|
+ onCompletion();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
protected void onCompletion() {
|
|
|
- if (task instanceof CancellableTask && ((CancellableTask) task).notifyIfCancelled(listener)) {
|
|
|
+ if ((task instanceof CancellableTask t) && t.notifyIfCancelled(listener)) {
|
|
|
return;
|
|
|
}
|
|
|
|
|
|
Response response = null;
|
|
|
try {
|
|
|
- response = newResponse(request, responses, unavailableShardCount, nodeIds, clusterState);
|
|
|
+ response = newResponse(request, nodeResponseTracker, unavailableShardCount, nodeIds, clusterState);
|
|
|
+ } catch (NodeResponseTracker.DiscardedResponsesException e) {
|
|
|
+ // We propagate the reason that the results, in this case the task cancellation, in case the listener needs to take
|
|
|
+ // follow-up actions
|
|
|
+ listener.onFailure((Exception) e.getCause());
|
|
|
} catch (Exception e) {
|
|
|
logger.debug("failed to combine responses from nodes", e);
|
|
|
listener.onFailure(e);
|
|
@@ -417,6 +416,21 @@ public abstract class TransportBroadcastByNodeAction<
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void onCancelled() {
|
|
|
+ assert task instanceof CancellableTask : "task must be cancellable";
|
|
|
+ try {
|
|
|
+ ((CancellableTask) task).ensureNotCancelled();
|
|
|
+ } catch (TaskCancelledException e) {
|
|
|
+ nodeResponseTracker.discardIntermediateResponses(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // For testing purposes
|
|
|
+ public NodeResponseTracker getNodeResponseTracker() {
|
|
|
+ return nodeResponseTracker;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
class BroadcastByNodeTransportRequestHandler implements TransportRequestHandler<NodeRequest> {
|