|  | @@ -20,41 +20,78 @@
 | 
	
		
			
				|  |  |  package org.elasticsearch.cluster.action.shard;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  import org.apache.lucene.index.CorruptIndexException;
 | 
	
		
			
				|  |  | +import org.elasticsearch.cluster.ClusterService;
 | 
	
		
			
				|  |  |  import org.elasticsearch.cluster.ClusterState;
 | 
	
		
			
				|  |  | -import org.elasticsearch.cluster.node.DiscoveryNode;
 | 
	
		
			
				|  |  | +import org.elasticsearch.cluster.ClusterStateObserver;
 | 
	
		
			
				|  |  | +import org.elasticsearch.cluster.NotMasterException;
 | 
	
		
			
				|  |  |  import org.elasticsearch.cluster.node.DiscoveryNodes;
 | 
	
		
			
				|  |  |  import org.elasticsearch.cluster.routing.IndexRoutingTable;
 | 
	
		
			
				|  |  | +import org.elasticsearch.cluster.routing.RoutingService;
 | 
	
		
			
				|  |  |  import org.elasticsearch.cluster.routing.ShardRouting;
 | 
	
		
			
				|  |  |  import org.elasticsearch.cluster.routing.ShardsIterator;
 | 
	
		
			
				|  |  | +import org.elasticsearch.cluster.routing.allocation.AllocationService;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.settings.Settings;
 | 
	
		
			
				|  |  | -import org.elasticsearch.common.unit.TimeValue;
 | 
	
		
			
				|  |  | +import org.elasticsearch.discovery.Discovery;
 | 
	
		
			
				|  |  |  import org.elasticsearch.test.ESTestCase;
 | 
	
		
			
				|  |  |  import org.elasticsearch.test.cluster.TestClusterService;
 | 
	
		
			
				|  |  |  import org.elasticsearch.test.transport.CapturingTransport;
 | 
	
		
			
				|  |  |  import org.elasticsearch.threadpool.ThreadPool;
 | 
	
		
			
				|  |  | -import org.elasticsearch.transport.ReceiveTimeoutTransportException;
 | 
	
		
			
				|  |  | +import org.elasticsearch.transport.NodeDisconnectedException;
 | 
	
		
			
				|  |  |  import org.elasticsearch.transport.TransportException;
 | 
	
		
			
				|  |  | +import org.elasticsearch.transport.TransportResponse;
 | 
	
		
			
				|  |  |  import org.elasticsearch.transport.TransportService;
 | 
	
		
			
				|  |  |  import org.junit.After;
 | 
	
		
			
				|  |  |  import org.junit.AfterClass;
 | 
	
		
			
				|  |  |  import org.junit.Before;
 | 
	
		
			
				|  |  |  import org.junit.BeforeClass;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +import java.util.ArrayList;
 | 
	
		
			
				|  |  | +import java.util.List;
 | 
	
		
			
				|  |  |  import java.util.concurrent.CountDownLatch;
 | 
	
		
			
				|  |  |  import java.util.concurrent.TimeUnit;
 | 
	
		
			
				|  |  |  import java.util.concurrent.atomic.AtomicBoolean;
 | 
	
		
			
				|  |  | +import java.util.concurrent.atomic.AtomicInteger;
 | 
	
		
			
				|  |  | +import java.util.concurrent.atomic.AtomicReference;
 | 
	
		
			
				|  |  | +import java.util.function.LongConsumer;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.stateWithStartedPrimary;
 | 
	
		
			
				|  |  |  import static org.hamcrest.CoreMatchers.equalTo;
 | 
	
		
			
				|  |  | +import static org.hamcrest.CoreMatchers.instanceOf;
 | 
	
		
			
				|  |  | +import static org.hamcrest.Matchers.is;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  public class ShardStateActionTests extends ESTestCase {
 | 
	
		
			
				|  |  |      private static ThreadPool THREAD_POOL;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    private ShardStateAction shardStateAction;
 | 
	
		
			
				|  |  | +    private TestShardStateAction shardStateAction;
 | 
	
		
			
				|  |  |      private CapturingTransport transport;
 | 
	
		
			
				|  |  |      private TransportService transportService;
 | 
	
		
			
				|  |  |      private TestClusterService clusterService;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +    private static class TestShardStateAction extends ShardStateAction {
 | 
	
		
			
				|  |  | +        public TestShardStateAction(Settings settings, ClusterService clusterService, TransportService transportService, AllocationService allocationService, RoutingService routingService) {
 | 
	
		
			
				|  |  | +            super(settings, clusterService, transportService, allocationService, routingService);
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        private Runnable onBeforeWaitForNewMasterAndRetry;
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        public void setOnBeforeWaitForNewMasterAndRetry(Runnable onBeforeWaitForNewMasterAndRetry) {
 | 
	
		
			
				|  |  | +            this.onBeforeWaitForNewMasterAndRetry = onBeforeWaitForNewMasterAndRetry;
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        private Runnable onAfterWaitForNewMasterAndRetry;
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        public void setOnAfterWaitForNewMasterAndRetry(Runnable onAfterWaitForNewMasterAndRetry) {
 | 
	
		
			
				|  |  | +            this.onAfterWaitForNewMasterAndRetry = onAfterWaitForNewMasterAndRetry;
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        @Override
 | 
	
		
			
				|  |  | +        protected void waitForNewMasterAndRetry(ClusterStateObserver observer, ShardRoutingEntry shardRoutingEntry, Listener listener) {
 | 
	
		
			
				|  |  | +            onBeforeWaitForNewMasterAndRetry.run();
 | 
	
		
			
				|  |  | +            super.waitForNewMasterAndRetry(observer, shardRoutingEntry, listener);
 | 
	
		
			
				|  |  | +            onAfterWaitForNewMasterAndRetry.run();
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |      @BeforeClass
 | 
	
		
			
				|  |  |      public static void startThreadPool() {
 | 
	
		
			
				|  |  |          THREAD_POOL = new ThreadPool("ShardStateActionTest");
 | 
	
	
		
			
				|  | @@ -68,7 +105,9 @@ public class ShardStateActionTests extends ESTestCase {
 | 
	
		
			
				|  |  |          clusterService = new TestClusterService(THREAD_POOL);
 | 
	
		
			
				|  |  |          transportService = new TransportService(transport, THREAD_POOL);
 | 
	
		
			
				|  |  |          transportService.start();
 | 
	
		
			
				|  |  | -        shardStateAction = new ShardStateAction(Settings.EMPTY, clusterService, transportService, null, null);
 | 
	
		
			
				|  |  | +        shardStateAction = new TestShardStateAction(Settings.EMPTY, clusterService, transportService, null, null);
 | 
	
		
			
				|  |  | +        shardStateAction.setOnBeforeWaitForNewMasterAndRetry(() -> {});
 | 
	
		
			
				|  |  | +        shardStateAction.setOnAfterWaitForNewMasterAndRetry(() -> {});
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      @Override
 | 
	
	
		
			
				|  | @@ -84,94 +123,165 @@ public class ShardStateActionTests extends ESTestCase {
 | 
	
		
			
				|  |  |          THREAD_POOL = null;
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    public void testNoMaster() {
 | 
	
		
			
				|  |  | +    public void testSuccess() throws InterruptedException {
 | 
	
		
			
				|  |  |          final String index = "test";
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          clusterService.setState(stateWithStartedPrimary(index, true, randomInt(5)));
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        DiscoveryNodes.Builder builder = DiscoveryNodes.builder(clusterService.state().nodes());
 | 
	
		
			
				|  |  | -        builder.masterNodeId(null);
 | 
	
		
			
				|  |  | -        clusterService.setState(ClusterState.builder(clusterService.state()).nodes(builder));
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  |          String indexUUID = clusterService.state().metaData().index(index).getIndexUUID();
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        AtomicBoolean noMaster = new AtomicBoolean();
 | 
	
		
			
				|  |  | -        assert !noMaster.get();
 | 
	
		
			
				|  |  | +        AtomicBoolean success = new AtomicBoolean();
 | 
	
		
			
				|  |  | +        CountDownLatch latch = new CountDownLatch(1);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        shardStateAction.shardFailed(clusterService.state(), getRandomShardRouting(index), indexUUID, "test", getSimulatedFailure(), new ShardStateAction.Listener() {
 | 
	
		
			
				|  |  | +        ShardRouting shardRouting = getRandomShardRouting(index);
 | 
	
		
			
				|  |  | +        shardStateAction.shardFailed(shardRouting, indexUUID, "test", getSimulatedFailure(), new ShardStateAction.Listener() {
 | 
	
		
			
				|  |  |              @Override
 | 
	
		
			
				|  |  | -            public void onShardFailedNoMaster() {
 | 
	
		
			
				|  |  | -                noMaster.set(true);
 | 
	
		
			
				|  |  | +            public void onSuccess() {
 | 
	
		
			
				|  |  | +                success.set(true);
 | 
	
		
			
				|  |  | +                latch.countDown();
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |              @Override
 | 
	
		
			
				|  |  | -            public void onShardFailedFailure(DiscoveryNode master, TransportException e) {
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | +            public void onShardFailedFailure(Exception e) {
 | 
	
		
			
				|  |  | +                success.set(false);
 | 
	
		
			
				|  |  | +                latch.countDown();
 | 
	
		
			
				|  |  | +                assert false;
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  |          });
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        assertTrue(noMaster.get());
 | 
	
		
			
				|  |  | +        CapturingTransport.CapturedRequest[] capturedRequests = transport.getCapturedRequestsAndClear();
 | 
	
		
			
				|  |  | +        assertEquals(1, capturedRequests.length);
 | 
	
		
			
				|  |  | +        // the request is a shard failed request
 | 
	
		
			
				|  |  | +        assertThat(capturedRequests[0].request, is(instanceOf(ShardStateAction.ShardRoutingEntry.class)));
 | 
	
		
			
				|  |  | +        ShardStateAction.ShardRoutingEntry shardRoutingEntry = (ShardStateAction.ShardRoutingEntry)capturedRequests[0].request;
 | 
	
		
			
				|  |  | +        // for the right shard
 | 
	
		
			
				|  |  | +        assertEquals(shardRouting, shardRoutingEntry.getShardRouting());
 | 
	
		
			
				|  |  | +        // sent to the master
 | 
	
		
			
				|  |  | +        assertEquals(clusterService.state().nodes().masterNode().getId(), capturedRequests[0].node.getId());
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        transport.handleResponse(capturedRequests[0].requestId, TransportResponse.Empty.INSTANCE);
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        latch.await();
 | 
	
		
			
				|  |  | +        assertTrue(success.get());
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    public void testFailure() {
 | 
	
		
			
				|  |  | +    public void testNoMaster() throws InterruptedException {
 | 
	
		
			
				|  |  |          final String index = "test";
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          clusterService.setState(stateWithStartedPrimary(index, true, randomInt(5)));
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +        DiscoveryNodes.Builder noMasterBuilder = DiscoveryNodes.builder(clusterService.state().nodes());
 | 
	
		
			
				|  |  | +        noMasterBuilder.masterNodeId(null);
 | 
	
		
			
				|  |  | +        clusterService.setState(ClusterState.builder(clusterService.state()).nodes(noMasterBuilder));
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |          String indexUUID = clusterService.state().metaData().index(index).getIndexUUID();
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        AtomicBoolean failure = new AtomicBoolean();
 | 
	
		
			
				|  |  | -        assert !failure.get();
 | 
	
		
			
				|  |  | +        CountDownLatch latch = new CountDownLatch(1);
 | 
	
		
			
				|  |  | +        AtomicInteger retries = new AtomicInteger();
 | 
	
		
			
				|  |  | +        AtomicBoolean success = new AtomicBoolean();
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        shardStateAction.shardFailed(clusterService.state(), getRandomShardRouting(index), indexUUID, "test", getSimulatedFailure(), new ShardStateAction.Listener() {
 | 
	
		
			
				|  |  | -            @Override
 | 
	
		
			
				|  |  | -            public void onShardFailedNoMaster() {
 | 
	
		
			
				|  |  | +        setUpMasterRetryVerification(1, retries, latch, requestId -> {});
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +        shardStateAction.shardFailed(getRandomShardRouting(index), indexUUID, "test", getSimulatedFailure(), new ShardStateAction.Listener() {
 | 
	
		
			
				|  |  | +            @Override
 | 
	
		
			
				|  |  | +            public void onSuccess() {
 | 
	
		
			
				|  |  | +                success.set(true);
 | 
	
		
			
				|  |  | +                latch.countDown();
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |              @Override
 | 
	
		
			
				|  |  | -            public void onShardFailedFailure(DiscoveryNode master, TransportException e) {
 | 
	
		
			
				|  |  | -                failure.set(true);
 | 
	
		
			
				|  |  | +            public void onShardFailedFailure(Exception e) {
 | 
	
		
			
				|  |  | +                success.set(false);
 | 
	
		
			
				|  |  | +                latch.countDown();
 | 
	
		
			
				|  |  | +                assert false;
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  |          });
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        final CapturingTransport.CapturedRequest[] capturedRequests = transport.getCapturedRequestsAndClear();
 | 
	
		
			
				|  |  | -        assertThat(capturedRequests.length, equalTo(1));
 | 
	
		
			
				|  |  | -        assert !failure.get();
 | 
	
		
			
				|  |  | -        transport.handleResponse(capturedRequests[0].requestId, new TransportException("simulated"));
 | 
	
		
			
				|  |  | +        latch.await();
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        assertTrue(failure.get());
 | 
	
		
			
				|  |  | +        assertThat(retries.get(), equalTo(1));
 | 
	
		
			
				|  |  | +        assertTrue(success.get());
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    public void testTimeout() throws InterruptedException {
 | 
	
		
			
				|  |  | +    public void testMasterChannelException() throws InterruptedException {
 | 
	
		
			
				|  |  |          final String index = "test";
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          clusterService.setState(stateWithStartedPrimary(index, true, randomInt(5)));
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          String indexUUID = clusterService.state().metaData().index(index).getIndexUUID();
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        AtomicBoolean progress = new AtomicBoolean();
 | 
	
		
			
				|  |  | -        AtomicBoolean timedOut = new AtomicBoolean();
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -        TimeValue timeout = new TimeValue(1, TimeUnit.MILLISECONDS);
 | 
	
		
			
				|  |  |          CountDownLatch latch = new CountDownLatch(1);
 | 
	
		
			
				|  |  | -        shardStateAction.shardFailed(clusterService.state(), getRandomShardRouting(index), indexUUID, "test", getSimulatedFailure(), timeout, new ShardStateAction.Listener() {
 | 
	
		
			
				|  |  | +        AtomicInteger retries = new AtomicInteger();
 | 
	
		
			
				|  |  | +        AtomicBoolean success = new AtomicBoolean();
 | 
	
		
			
				|  |  | +        AtomicReference<Exception> exception = new AtomicReference<>();
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        LongConsumer retryLoop = requestId -> {
 | 
	
		
			
				|  |  | +            List<Exception> possibleExceptions = new ArrayList<>();
 | 
	
		
			
				|  |  | +            possibleExceptions.add(new NotMasterException("simulated"));
 | 
	
		
			
				|  |  | +            possibleExceptions.add(new NodeDisconnectedException(clusterService.state().nodes().masterNode(), ShardStateAction.SHARD_FAILED_ACTION_NAME));
 | 
	
		
			
				|  |  | +            possibleExceptions.add(new Discovery.FailedToCommitClusterStateException("simulated"));
 | 
	
		
			
				|  |  | +            transport.handleResponse(requestId, randomFrom(possibleExceptions));
 | 
	
		
			
				|  |  | +        };
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        final int numberOfRetries = randomIntBetween(1, 256);
 | 
	
		
			
				|  |  | +        setUpMasterRetryVerification(numberOfRetries, retries, latch, retryLoop);
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        shardStateAction.shardFailed(getRandomShardRouting(index), indexUUID, "test", getSimulatedFailure(), new ShardStateAction.Listener() {
 | 
	
		
			
				|  |  | +            @Override
 | 
	
		
			
				|  |  | +            public void onSuccess() {
 | 
	
		
			
				|  |  | +                success.set(true);
 | 
	
		
			
				|  |  | +                latch.countDown();
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |              @Override
 | 
	
		
			
				|  |  | -            public void onShardFailedFailure(DiscoveryNode master, TransportException e) {
 | 
	
		
			
				|  |  | -                if (e instanceof ReceiveTimeoutTransportException) {
 | 
	
		
			
				|  |  | -                    assertFalse(progress.get());
 | 
	
		
			
				|  |  | -                    timedOut.set(true);
 | 
	
		
			
				|  |  | -                }
 | 
	
		
			
				|  |  | +            public void onShardFailedFailure(Exception e) {
 | 
	
		
			
				|  |  | +                success.set(false);
 | 
	
		
			
				|  |  | +                exception.set(e);
 | 
	
		
			
				|  |  |                  latch.countDown();
 | 
	
		
			
				|  |  | +                assert false;
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  |          });
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +        final CapturingTransport.CapturedRequest[] capturedRequests = transport.getCapturedRequestsAndClear();
 | 
	
		
			
				|  |  | +        assertThat(capturedRequests.length, equalTo(1));
 | 
	
		
			
				|  |  | +        assertFalse(success.get());
 | 
	
		
			
				|  |  | +        assertThat(retries.get(), equalTo(0));
 | 
	
		
			
				|  |  | +        retryLoop.accept(capturedRequests[0].requestId);
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |          latch.await();
 | 
	
		
			
				|  |  | -        progress.set(true);
 | 
	
		
			
				|  |  | -        assertTrue(timedOut.get());
 | 
	
		
			
				|  |  | +        assertNull(exception.get());
 | 
	
		
			
				|  |  | +        assertThat(retries.get(), equalTo(numberOfRetries));
 | 
	
		
			
				|  |  | +        assertTrue(success.get());
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +    public void testUnhandledFailure() {
 | 
	
		
			
				|  |  | +        final String index = "test";
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        clusterService.setState(stateWithStartedPrimary(index, true, randomInt(5)));
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        String indexUUID = clusterService.state().metaData().index(index).getIndexUUID();
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        AtomicBoolean failure = new AtomicBoolean();
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        shardStateAction.shardFailed(getRandomShardRouting(index), indexUUID, "test", getSimulatedFailure(), new ShardStateAction.Listener() {
 | 
	
		
			
				|  |  | +            @Override
 | 
	
		
			
				|  |  | +            public void onSuccess() {
 | 
	
		
			
				|  |  | +                failure.set(false);
 | 
	
		
			
				|  |  | +                assert false;
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +            @Override
 | 
	
		
			
				|  |  | +            public void onShardFailedFailure(Exception e) {
 | 
	
		
			
				|  |  | +                failure.set(true);
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  | +        });
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          final CapturingTransport.CapturedRequest[] capturedRequests = transport.getCapturedRequestsAndClear();
 | 
	
		
			
				|  |  |          assertThat(capturedRequests.length, equalTo(1));
 | 
	
		
			
				|  |  | +        assertFalse(failure.get());
 | 
	
		
			
				|  |  | +        transport.handleResponse(capturedRequests[0].requestId, new TransportException("simulated"));
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        assertTrue(failure.get());
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      private ShardRouting getRandomShardRouting(String index) {
 | 
	
	
		
			
				|  | @@ -182,6 +292,34 @@ public class ShardStateActionTests extends ESTestCase {
 | 
	
		
			
				|  |  |          return shardRouting;
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +    private void setUpMasterRetryVerification(int numberOfRetries, AtomicInteger retries, CountDownLatch latch, LongConsumer retryLoop) {
 | 
	
		
			
				|  |  | +        shardStateAction.setOnBeforeWaitForNewMasterAndRetry(() -> {
 | 
	
		
			
				|  |  | +            DiscoveryNodes.Builder masterBuilder = DiscoveryNodes.builder(clusterService.state().nodes());
 | 
	
		
			
				|  |  | +            masterBuilder.masterNodeId(clusterService.state().nodes().masterNodes().iterator().next().value.id());
 | 
	
		
			
				|  |  | +            clusterService.setState(ClusterState.builder(clusterService.state()).nodes(masterBuilder));
 | 
	
		
			
				|  |  | +        });
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        shardStateAction.setOnAfterWaitForNewMasterAndRetry(() -> verifyRetry(numberOfRetries, retries, latch, retryLoop));
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +    private void verifyRetry(int numberOfRetries, AtomicInteger retries, CountDownLatch latch, LongConsumer retryLoop) {
 | 
	
		
			
				|  |  | +        // assert a retry request was sent
 | 
	
		
			
				|  |  | +        final CapturingTransport.CapturedRequest[] capturedRequests = transport.getCapturedRequestsAndClear();
 | 
	
		
			
				|  |  | +        if (capturedRequests.length == 1) {
 | 
	
		
			
				|  |  | +            retries.incrementAndGet();
 | 
	
		
			
				|  |  | +            if (retries.get() == numberOfRetries) {
 | 
	
		
			
				|  |  | +                // finish the request
 | 
	
		
			
				|  |  | +                transport.handleResponse(capturedRequests[0].requestId, TransportResponse.Empty.INSTANCE);
 | 
	
		
			
				|  |  | +            } else {
 | 
	
		
			
				|  |  | +                retryLoop.accept(capturedRequests[0].requestId);
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  | +        } else {
 | 
	
		
			
				|  |  | +            // there failed to be a retry request
 | 
	
		
			
				|  |  | +            // release the driver thread to fail the test
 | 
	
		
			
				|  |  | +            latch.countDown();
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |      private Throwable getSimulatedFailure() {
 | 
	
		
			
				|  |  |          return new CorruptIndexException("simulated", (String) null);
 | 
	
		
			
				|  |  |      }
 |