84
84
import java .util .concurrent .atomic .AtomicBoolean ;
85
85
import java .util .concurrent .atomic .AtomicInteger ;
86
86
import java .util .concurrent .atomic .AtomicReference ;
87
+ import java .util .function .Consumer ;
87
88
88
89
import static org .elasticsearch .action .support .replication .ClusterStateCreationUtils .state ;
89
90
import static org .elasticsearch .action .support .replication .ClusterStateCreationUtils .stateWithActivePrimary ;
91
+ import static org .hamcrest .CoreMatchers .containsString ;
90
92
import static org .hamcrest .CoreMatchers .not ;
91
93
import static org .hamcrest .Matchers .arrayWithSize ;
92
94
import static org .hamcrest .Matchers .either ;
@@ -631,9 +633,11 @@ protected void runReplicateTest(ClusterState state, IndexShardRoutingTable shard
631
633
indexShardRouting .set (primaryShard );
632
634
633
635
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 );
637
641
638
642
assertThat (replicationPhase .totalShards (), equalTo (totalShards ));
639
643
assertThat (replicationPhase .pending (), equalTo (assignedReplicas ));
@@ -704,7 +708,8 @@ protected void runReplicateTest(ClusterState state, IndexShardRoutingTable shard
704
708
// the shard the request was sent to and the shard to be failed should be the same
705
709
assertEquals (shardRoutingEntry .getShardRouting (), routing );
706
710
failures .add (shardFailedRequest );
707
- if (randomBoolean ()) {
711
+ int ternary = randomIntBetween (0 , 2 );
712
+ if (ternary == 0 ) {
708
713
// simulate master left and test that the shard failure is retried
709
714
int numberOfRetries = randomIntBetween (1 , 4 );
710
715
CapturingTransport .CapturedRequest currentRequest = shardFailedRequest ;
@@ -718,8 +723,19 @@ protected void runReplicateTest(ClusterState state, IndexShardRoutingTable shard
718
723
}
719
724
// now simulate that the last retry succeeded
720
725
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 ) {
722
736
transport .handleResponse (shardFailedRequest .requestId , TransportResponse .Empty .INSTANCE );
737
+ } else {
738
+ assert false ;
723
739
}
724
740
}
725
741
} else {
@@ -882,14 +898,85 @@ public void testCounterDecrementedIfShardOperationThrowsException() throws Inter
882
898
assertPhase (task , "failed" );
883
899
}
884
900
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
+
885
966
private void assertIndexShardCounter (int expected ) {
886
967
assertThat (count .get (), equalTo (expected ));
887
968
}
888
969
970
+ private void assertShardIsFailed () {
971
+ assertTrue (isShardFailed .get ());
972
+ }
973
+
889
974
private final AtomicInteger count = new AtomicInteger (0 );
890
975
891
976
private final AtomicBoolean isRelocated = new AtomicBoolean (false );
892
977
978
+ private final AtomicBoolean isShardFailed = new AtomicBoolean ();
979
+
893
980
private final AtomicReference <ShardRouting > indexShardRouting = new AtomicReference <>();
894
981
895
982
/**
@@ -903,6 +990,11 @@ public boolean isRelocated() {
903
990
return isRelocated .get ();
904
991
}
905
992
993
+ @ Override
994
+ public void failShard (String reason , @ Nullable Throwable e ) {
995
+ isShardFailed .set (true );
996
+ }
997
+
906
998
@ Override
907
999
public ShardRouting routingEntry () {
908
1000
ShardRouting shardRouting = indexShardRouting .get ();
@@ -1099,6 +1191,10 @@ protected void shardOperationOnReplica(Request shardRequest) {
1099
1191
* Transport channel that is needed for replica operation testing.
1100
1192
*/
1101
1193
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 ) {
1102
1198
return new TransportChannel () {
1103
1199
1104
1200
@ Override
@@ -1123,6 +1219,7 @@ public void sendResponse(TransportResponse response, TransportResponseOptions op
1123
1219
1124
1220
@ Override
1125
1221
public void sendResponse (Throwable error ) throws IOException {
1222
+ consumer .accept (error );
1126
1223
listener .onFailure (error );
1127
1224
}
1128
1225
0 commit comments