Skip to content

Commit 346ff04

Browse files
committed
Fail demoted primary shards and retry request
This commit handles the scenario where a replication action fails on a replica shard, the primary shard attempts to fail the replica shard but the primary shard is notified of demotion by the master. In this scenario, the demoted primary shard must be failed, and then the request rerouted again to the new primary shard. Closes #16415, closes #14252
1 parent 321c463 commit 346ff04

File tree

3 files changed

+128
-11
lines changed

3 files changed

+128
-11
lines changed

core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java

+23-5
Original file line numberDiff line numberDiff line change
@@ -415,7 +415,11 @@ protected void doRun() throws Exception {
415415

416416
public static class RetryOnPrimaryException extends ElasticsearchException {
417417
public RetryOnPrimaryException(ShardId shardId, String msg) {
418-
super(msg);
418+
this(shardId, msg, null);
419+
}
420+
421+
public RetryOnPrimaryException(ShardId shardId, String msg, Throwable cause) {
422+
super(msg, cause);
419423
setShard(shardId);
420424
}
421425

@@ -801,6 +805,7 @@ protected IndexShardReference getIndexShardReferenceOnReplica(ShardId shardId) {
801805
* relocating copies
802806
*/
803807
final class ReplicationPhase extends AbstractRunnable {
808+
804809
private final ReplicationTask task;
805810
private final ReplicaRequest replicaRequest;
806811
private final Response finalResponse;
@@ -982,9 +987,17 @@ public void onSuccess() {
982987
}
983988

984989
@Override
985-
public void onFailure(Throwable t) {
986-
// TODO: handle catastrophic non-channel failures
987-
onReplicaFailure(nodeId, exp);
990+
public void onFailure(Throwable shardFailedError) {
991+
if (shardFailedError instanceof ShardStateAction.NoLongerPrimaryShardException) {
992+
ShardRouting primaryShard = indexShardReference.routingEntry();
993+
String message = String.format(Locale.ROOT, "primary shard [%s] was demoted while failing replica shard [%s] for [%s]", primaryShard, shard, exp);
994+
// we are no longer the primary, fail ourselves and start over
995+
indexShardReference.failShard(message, shardFailedError);
996+
forceFinishAsFailed(new RetryOnPrimaryException(shardId, message, shardFailedError));
997+
} else {
998+
assert false : shardFailedError;
999+
onReplicaFailure(nodeId, exp);
1000+
}
9881001
}
9891002
}
9901003
);
@@ -1070,7 +1083,7 @@ protected boolean shouldExecuteReplication(Settings settings) {
10701083

10711084
interface IndexShardReference extends Releasable {
10721085
boolean isRelocated();
1073-
1086+
void failShard(String reason, @Nullable Throwable e);
10741087
ShardRouting routingEntry();
10751088
}
10761089

@@ -1098,6 +1111,11 @@ public boolean isRelocated() {
10981111
return indexShard.state() == IndexShardState.RELOCATED;
10991112
}
11001113

1114+
@Override
1115+
public void failShard(String reason, @Nullable Throwable e) {
1116+
indexShard.failShard(reason, e);
1117+
}
1118+
11011119
@Override
11021120
public ShardRouting routingEntry() {
11031121
return indexShard.routingEntry();

core/src/test/java/org/elasticsearch/action/support/replication/ClusterStateCreationUtils.java

+3-1
Original file line numberDiff line numberDiff line change
@@ -101,7 +101,9 @@ public static ClusterState state(String index, boolean activePrimaryLocal, Shard
101101
primaryNode = newNode(0).id();
102102
unassignedNodes.remove(primaryNode);
103103
} else {
104-
primaryNode = selectAndRemove(unassignedNodes);
104+
Set<String> unassignedNodesExecludingPrimary = new HashSet<>(unassignedNodes);
105+
unassignedNodesExecludingPrimary.remove(newNode(0).id());
106+
primaryNode = selectAndRemove(unassignedNodesExecludingPrimary);
105107
}
106108
if (primaryState == ShardRoutingState.RELOCATING) {
107109
relocatingNode = selectAndRemove(unassignedNodes);

core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java

+102-5
Original file line numberDiff line numberDiff line change
@@ -84,9 +84,11 @@
8484
import java.util.concurrent.atomic.AtomicBoolean;
8585
import java.util.concurrent.atomic.AtomicInteger;
8686
import java.util.concurrent.atomic.AtomicReference;
87+
import java.util.function.Consumer;
8788

8889
import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.state;
8990
import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.stateWithActivePrimary;
91+
import static org.hamcrest.CoreMatchers.containsString;
9092
import static org.hamcrest.CoreMatchers.not;
9193
import static org.hamcrest.Matchers.arrayWithSize;
9294
import static org.hamcrest.Matchers.either;
@@ -631,9 +633,11 @@ protected void runReplicateTest(ClusterState state, IndexShardRoutingTable shard
631633
indexShardRouting.set(primaryShard);
632634

633635
assertIndexShardCounter(2);
634-
// TODO: set a default timeout
635-
TransportReplicationAction<Request, Request, Response>.ReplicationPhase replicationPhase = action.new ReplicationPhase(task,
636-
request, new Response(), request.shardId(), createTransportChannel(listener), reference);
636+
AtomicReference<Throwable> error = new AtomicReference<>();
637+
638+
TransportChannel channel = createTransportChannel(listener, error::set);
639+
TransportReplicationAction<Request, Request, Response>.ReplicationPhase replicationPhase =
640+
action.new ReplicationPhase(task, request, new Response(), request.shardId(), channel, reference);
637641

638642
assertThat(replicationPhase.totalShards(), equalTo(totalShards));
639643
assertThat(replicationPhase.pending(), equalTo(assignedReplicas));
@@ -704,7 +708,8 @@ protected void runReplicateTest(ClusterState state, IndexShardRoutingTable shard
704708
// the shard the request was sent to and the shard to be failed should be the same
705709
assertEquals(shardRoutingEntry.getShardRouting(), routing);
706710
failures.add(shardFailedRequest);
707-
if (randomBoolean()) {
711+
int ternary = randomIntBetween(0, 2);
712+
if (ternary == 0) {
708713
// simulate master left and test that the shard failure is retried
709714
int numberOfRetries = randomIntBetween(1, 4);
710715
CapturingTransport.CapturedRequest currentRequest = shardFailedRequest;
@@ -718,8 +723,19 @@ protected void runReplicateTest(ClusterState state, IndexShardRoutingTable shard
718723
}
719724
// now simulate that the last retry succeeded
720725
transport.handleResponse(currentRequest.requestId, TransportResponse.Empty.INSTANCE);
721-
} else {
726+
} else if (ternary == 1) {
727+
// simulate the primary has been demoted
728+
transport.handleRemoteError(shardFailedRequest.requestId, new ShardStateAction.NoLongerPrimaryShardException(shardRoutingEntry.getShardRouting().shardId(), "shard-failed-test"));
729+
// the primary should fail itself
730+
assertShardIsFailed();
731+
// we should see a retry on primary exception
732+
assertNotNull(error.get());
733+
assertThat(error.get(), instanceOf(TransportReplicationAction.RetryOnPrimaryException.class));
734+
return;
735+
} else if (ternary == 2) {
722736
transport.handleResponse(shardFailedRequest.requestId, TransportResponse.Empty.INSTANCE);
737+
} else {
738+
assert false;
723739
}
724740
}
725741
} else {
@@ -882,14 +898,85 @@ public void testCounterDecrementedIfShardOperationThrowsException() throws Inter
882898
assertPhase(task, "failed");
883899
}
884900

901+
public void testReroutePhaseRetriedAfterDemotedPrimary() {
902+
final String index = "test";
903+
final ShardId shardId = new ShardId(index, "_na_", 0);
904+
boolean localPrimary = true;
905+
clusterService.setState(state(index, localPrimary,
906+
ShardRoutingState.STARTED, ShardRoutingState.STARTED));
907+
Action action = new Action(Settings.EMPTY, "testAction", transportService, clusterService, threadPool) {
908+
@Override
909+
protected void resolveRequest(MetaData metaData, String concreteIndex, Request request) {
910+
request.setShardId(shardId);
911+
}
912+
};
913+
Request request = new Request();
914+
PlainActionFuture<Response> listener = new PlainActionFuture<>();
915+
916+
TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(null, request, listener);
917+
reroutePhase.run();
918+
919+
// reroute phase should send primary action
920+
CapturingTransport.CapturedRequest[] primaryRequests = transport.getCapturedRequestsAndClear();
921+
assertThat(primaryRequests.length, equalTo(1));
922+
assertThat(primaryRequests[0].action, equalTo("testAction" + (localPrimary ? "[p]" : "")));
923+
AtomicReference<Throwable> error = new AtomicReference<>();
924+
TransportChannel channel = createTransportChannel(listener, error::set);
925+
926+
// simulate primary action
927+
TransportReplicationAction.PrimaryPhase primaryPhase = action.new PrimaryPhase(maybeTask(), request, channel);
928+
primaryPhase.run();
929+
930+
// primary action should send replica request
931+
CapturingTransport.CapturedRequest[] replicaRequests = transport.getCapturedRequestsAndClear();
932+
assertThat(replicaRequests.length, equalTo(1));
933+
assertThat(replicaRequests[0].action, equalTo("testAction[r]"));
934+
indexShardRouting.set(clusterService.state().getRoutingTable().shardRoutingTable(shardId).primaryShard());
935+
936+
// simulate replica failure
937+
transport.handleRemoteError(replicaRequests[0].requestId, new Exception("exception"));
938+
939+
// the primary should request replica failure
940+
CapturingTransport.CapturedRequest[] replicaFailures = transport.getCapturedRequestsAndClear();
941+
assertThat(replicaFailures.length, equalTo(1));
942+
assertThat(replicaFailures[0].action, equalTo(ShardStateAction.SHARD_FAILED_ACTION_NAME));
943+
944+
// simulate demoted primary
945+
transport.handleRemoteError(replicaFailures[0].requestId, new ShardStateAction.NoLongerPrimaryShardException(shardId, "demoted"));
946+
assertTrue(isShardFailed.get());
947+
assertTrue(listener.isDone());
948+
assertNotNull(error.get());
949+
assertThat(error.get(), instanceOf(TransportReplicationAction.RetryOnPrimaryException.class));
950+
assertThat(error.get().getMessage(), containsString("was demoted while failing replica shard"));
951+
952+
// reroute phase sees the retry
953+
transport.handleRemoteError(primaryRequests[0].requestId, error.get());
954+
955+
// publish a new cluster state
956+
boolean localPrimaryOnRetry = randomBoolean();
957+
clusterService.setState(state(index, localPrimaryOnRetry,
958+
ShardRoutingState.STARTED, ShardRoutingState.STARTED));
959+
CapturingTransport.CapturedRequest[] primaryRetry = transport.getCapturedRequestsAndClear();
960+
961+
// the request should be retried
962+
assertThat(primaryRetry.length, equalTo(1));
963+
assertThat(primaryRetry[0].action, equalTo("testAction" + (localPrimaryOnRetry ? "[p]" : "")));
964+
}
965+
885966
private void assertIndexShardCounter(int expected) {
886967
assertThat(count.get(), equalTo(expected));
887968
}
888969

970+
private void assertShardIsFailed() {
971+
assertTrue(isShardFailed.get());
972+
}
973+
889974
private final AtomicInteger count = new AtomicInteger(0);
890975

891976
private final AtomicBoolean isRelocated = new AtomicBoolean(false);
892977

978+
private final AtomicBoolean isShardFailed = new AtomicBoolean();
979+
893980
private final AtomicReference<ShardRouting> indexShardRouting = new AtomicReference<>();
894981

895982
/**
@@ -903,6 +990,11 @@ public boolean isRelocated() {
903990
return isRelocated.get();
904991
}
905992

993+
@Override
994+
public void failShard(String reason, @Nullable Throwable e) {
995+
isShardFailed.set(true);
996+
}
997+
906998
@Override
907999
public ShardRouting routingEntry() {
9081000
ShardRouting shardRouting = indexShardRouting.get();
@@ -1099,6 +1191,10 @@ protected void shardOperationOnReplica(Request shardRequest) {
10991191
* Transport channel that is needed for replica operation testing.
11001192
*/
11011193
public TransportChannel createTransportChannel(final PlainActionFuture<Response> listener) {
1194+
return createTransportChannel(listener, error -> {});
1195+
}
1196+
1197+
public TransportChannel createTransportChannel(final PlainActionFuture<Response> listener, Consumer<Throwable> consumer) {
11021198
return new TransportChannel() {
11031199

11041200
@Override
@@ -1123,6 +1219,7 @@ public void sendResponse(TransportResponse response, TransportResponseOptions op
11231219

11241220
@Override
11251221
public void sendResponse(Throwable error) throws IOException {
1222+
consumer.accept(error);
11261223
listener.onFailure(error);
11271224
}
11281225

0 commit comments

Comments
 (0)