Răsfoiți Sursa

Replication operation that try to perform the primary phase on a replica should be retried

In extreme cases a local primary shard can be replaced with a replica while a replication request is in flight and the primary action is applied to the shard (via `acquirePrimaryOperationLock()).  #17044 changed the exception used in that method to something that isn't recognized as `TransportActions.isShardNotAvailableException`, causing the operation to fail immediately instead of retrying. This commit fixes this by check the primary flag before
acquiring the lock. This is safe to do as an IndexShard will never be demoted once a primary.

Closes #17358
Boaz Leskes 9 ani în urmă
părinte
comite
48b4f086e0

+ 9 - 2
core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java

@@ -549,8 +549,9 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
                 public void handleException(TransportException exp) {
                 public void handleException(TransportException exp) {
                     try {
                     try {
                         // if we got disconnected from the node, or the node / shard is not in the right state (being closed)
                         // if we got disconnected from the node, or the node / shard is not in the right state (being closed)
-                        if (exp.unwrapCause() instanceof ConnectTransportException || exp.unwrapCause() instanceof NodeClosedException ||
-                                (isPrimaryAction && retryPrimaryException(exp.unwrapCause()))) {
+                        final Throwable cause = exp.unwrapCause();
+                        if (cause instanceof ConnectTransportException || cause instanceof NodeClosedException ||
+                            (isPrimaryAction && retryPrimaryException(cause))) {
                             logger.trace("received an error from node [{}] for request [{}], scheduling a retry", exp, node.id(), request);
                             logger.trace("received an error from node [{}] for request [{}], scheduling a retry", exp, node.id(), request);
                             retry(exp);
                             retry(exp);
                         } else {
                         } else {
@@ -799,6 +800,12 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
     protected IndexShardReference getIndexShardReferenceOnPrimary(ShardId shardId) {
     protected IndexShardReference getIndexShardReferenceOnPrimary(ShardId shardId) {
         IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
         IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
         IndexShard indexShard = indexService.getShard(shardId.id());
         IndexShard indexShard = indexService.getShard(shardId.id());
+        // we may end up here if the cluster state used to route the primary is so stale that the underlying
+        // index shard was replaced with a replica. For example - in a two node cluster, if the primary fails
+        // the replica will take over and a replica will be assigned to the first node.
+        if (indexShard.routingEntry().primary() == false) {
+            throw new RetryOnPrimaryException(indexShard.shardId(), "actual shard is not a primary " +  indexShard.routingEntry());
+        }
         return IndexShardReferenceImpl.createOnPrimary(indexShard);
         return IndexShardReferenceImpl.createOnPrimary(indexShard);
     }
     }
 
 

+ 0 - 1
core/src/main/java/org/elasticsearch/index/shard/IndexShard.java

@@ -974,7 +974,6 @@ public class IndexShard extends AbstractIndexShardComponent {
 
 
     private void verifyPrimary() {
     private void verifyPrimary() {
         if (shardRouting.primary() == false) {
         if (shardRouting.primary() == false) {
-            // must use exception that is not ignored by replication logic. See TransportActions.isShardNotAvailableException
             throw new IllegalStateException("shard is not a primary " + shardRouting);
             throw new IllegalStateException("shard is not a primary " + shardRouting);
         }
         }
     }
     }

+ 78 - 21
core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java

@@ -50,6 +50,8 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.index.IndexNotFoundException;
 import org.elasticsearch.index.IndexNotFoundException;
+import org.elasticsearch.index.engine.EngineClosedException;
+import org.elasticsearch.index.shard.IndexShardClosedException;
 import org.elasticsearch.index.shard.IndexShardNotStartedException;
 import org.elasticsearch.index.shard.IndexShardNotStartedException;
 import org.elasticsearch.index.shard.IndexShardState;
 import org.elasticsearch.index.shard.IndexShardState;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.index.shard.ShardId;
@@ -158,7 +160,7 @@ public class TransportReplicationActionTests extends ESTestCase {
         ReplicationTask task = maybeTask();
         ReplicationTask task = maybeTask();
 
 
         ClusterBlocks.Builder block = ClusterBlocks.builder()
         ClusterBlocks.Builder block = ClusterBlocks.builder()
-                .addGlobalBlock(new ClusterBlock(1, "non retryable", false, true, RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL));
+            .addGlobalBlock(new ClusterBlock(1, "non retryable", false, true, RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL));
         setState(clusterService, ClusterState.builder(clusterService.state()).blocks(block));
         setState(clusterService, ClusterState.builder(clusterService.state()).blocks(block));
         TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
         TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
         reroutePhase.run();
         reroutePhase.run();
@@ -166,7 +168,7 @@ public class TransportReplicationActionTests extends ESTestCase {
         assertPhase(task, "failed");
         assertPhase(task, "failed");
 
 
         block = ClusterBlocks.builder()
         block = ClusterBlocks.builder()
-                .addGlobalBlock(new ClusterBlock(1, "retryable", true, true, RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL));
+            .addGlobalBlock(new ClusterBlock(1, "retryable", true, true, RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL));
         setState(clusterService, ClusterState.builder(clusterService.state()).blocks(block));
         setState(clusterService, ClusterState.builder(clusterService.state()).blocks(block));
         listener = new PlainActionFuture<>();
         listener = new PlainActionFuture<>();
         reroutePhase = action.new ReroutePhase(task, new Request().timeout("5ms"), listener);
         reroutePhase = action.new ReroutePhase(task, new Request().timeout("5ms"), listener);
@@ -181,7 +183,7 @@ public class TransportReplicationActionTests extends ESTestCase {
         assertPhase(task, "waiting_for_retry");
         assertPhase(task, "waiting_for_retry");
 
 
         block = ClusterBlocks.builder()
         block = ClusterBlocks.builder()
-                .addGlobalBlock(new ClusterBlock(1, "non retryable", false, true, RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL));
+            .addGlobalBlock(new ClusterBlock(1, "non retryable", false, true, RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL));
         setState(clusterService, ClusterState.builder(clusterService.state()).blocks(block));
         setState(clusterService, ClusterState.builder(clusterService.state()).blocks(block));
         assertListenerThrows("primary phase should fail operation when moving from a retryable block to a non-retryable one", listener, ClusterBlockException.class);
         assertListenerThrows("primary phase should fail operation when moving from a retryable block to a non-retryable one", listener, ClusterBlockException.class);
         assertIndexShardUninitialized();
         assertIndexShardUninitialized();
@@ -196,7 +198,7 @@ public class TransportReplicationActionTests extends ESTestCase {
         final ShardId shardId = new ShardId(index, "_na_", 0);
         final ShardId shardId = new ShardId(index, "_na_", 0);
         // no replicas in oder to skip the replication part
         // no replicas in oder to skip the replication part
         setState(clusterService, state(index, true,
         setState(clusterService, state(index, true,
-                randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED));
+            randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED));
         ReplicationTask task = maybeTask();
         ReplicationTask task = maybeTask();
 
 
         logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint());
         logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint());
@@ -221,7 +223,7 @@ public class TransportReplicationActionTests extends ESTestCase {
         final IndexShardRoutingTable shardRoutingTable = clusterService.state().routingTable().index(index).shard(shardId.id());
         final IndexShardRoutingTable shardRoutingTable = clusterService.state().routingTable().index(index).shard(shardId.id());
         final String primaryNodeId = shardRoutingTable.primaryShard().currentNodeId();
         final String primaryNodeId = shardRoutingTable.primaryShard().currentNodeId();
         final List<CapturingTransport.CapturedRequest> capturedRequests =
         final List<CapturingTransport.CapturedRequest> capturedRequests =
-                transport.getCapturedRequestsByTargetNodeAndClear().get(primaryNodeId);
+            transport.getCapturedRequestsByTargetNodeAndClear().get(primaryNodeId);
         assertThat(capturedRequests, notNullValue());
         assertThat(capturedRequests, notNullValue());
         assertThat(capturedRequests.size(), equalTo(1));
         assertThat(capturedRequests.size(), equalTo(1));
         assertThat(capturedRequests.get(0).action, equalTo("testAction[p]"));
         assertThat(capturedRequests.get(0).action, equalTo("testAction[p]"));
@@ -234,7 +236,7 @@ public class TransportReplicationActionTests extends ESTestCase {
      * before the relocation target, there is a time span where relocation source believes active primary to be on
      * before the relocation target, there is a time span where relocation source believes active primary to be on
      * relocation target and relocation target believes active primary to be on relocation source. This results in replication
      * relocation target and relocation target believes active primary to be on relocation source. This results in replication
      * requests being sent back and forth.
      * requests being sent back and forth.
-     *
+     * <p>
      * This test checks that replication request is not routed back from relocation target to relocation source in case of
      * This test checks that replication request is not routed back from relocation target to relocation source in case of
      * stale index routing table on relocation target.
      * stale index routing table on relocation target.
      */
      */
@@ -271,7 +273,7 @@ public class TransportReplicationActionTests extends ESTestCase {
         IndexShardRoutingTable shardRoutingTable = clusterService.state().routingTable().index(index).shard(shardId.id());
         IndexShardRoutingTable shardRoutingTable = clusterService.state().routingTable().index(index).shard(shardId.id());
         final String primaryNodeId = shardRoutingTable.primaryShard().currentNodeId();
         final String primaryNodeId = shardRoutingTable.primaryShard().currentNodeId();
         final List<CapturingTransport.CapturedRequest> capturedRequests =
         final List<CapturingTransport.CapturedRequest> capturedRequests =
-                transport.getCapturedRequestsByTargetNodeAndClear().get(primaryNodeId);
+            transport.getCapturedRequestsByTargetNodeAndClear().get(primaryNodeId);
         assertThat(capturedRequests, notNullValue());
         assertThat(capturedRequests, notNullValue());
         assertThat(capturedRequests.size(), equalTo(1));
         assertThat(capturedRequests.size(), equalTo(1));
         assertThat(capturedRequests.get(0).action, equalTo("testAction[p]"));
         assertThat(capturedRequests.get(0).action, equalTo("testAction[p]"));
@@ -282,7 +284,7 @@ public class TransportReplicationActionTests extends ESTestCase {
         final String index = "test";
         final String index = "test";
         // no replicas in oder to skip the replication part
         // no replicas in oder to skip the replication part
         setState(clusterService, state(index, true,
         setState(clusterService, state(index, true,
-                randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED));
+            randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED));
         logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint());
         logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint());
         Request request = new Request(new ShardId("unknown_index", "_na_", 0)).timeout("1ms");
         Request request = new Request(new ShardId("unknown_index", "_na_", 0)).timeout("1ms");
         PlainActionFuture<Response> listener = new PlainActionFuture<>();
         PlainActionFuture<Response> listener = new PlainActionFuture<>();
@@ -299,6 +301,61 @@ public class TransportReplicationActionTests extends ESTestCase {
         assertListenerThrows("must throw shard not found exception", listener, ShardNotFoundException.class);
         assertListenerThrows("must throw shard not found exception", listener, ShardNotFoundException.class);
     }
     }
 
 
+    public void testStalePrimaryShardOnReroute() throws InterruptedException {
+        final String index = "test";
+        final ShardId shardId = new ShardId(index, "_na_", 0);
+        // no replicas in order to skip the replication part
+        setState(clusterService, stateWithActivePrimary(index, true, randomInt(3)));
+        logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint());
+        Request request = new Request(shardId);
+        boolean timeout = randomBoolean();
+        if (timeout) {
+            request.timeout("0s");
+        } else {
+            request.timeout("1h");
+        }
+        PlainActionFuture<Response> listener = new PlainActionFuture<>();
+        ReplicationTask task = maybeTask();
+
+        TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
+        reroutePhase.run();
+        CapturingTransport.CapturedRequest[] capturedRequests = transport.getCapturedRequestsAndClear();
+        assertThat(capturedRequests, arrayWithSize(1));
+        assertThat(capturedRequests[0].action, equalTo("testAction[p]"));
+        assertPhase(task, "waiting_on_primary");
+        transport.handleRemoteError(capturedRequests[0].requestId, randomRetryPrimaryException(shardId));
+
+
+        if (timeout) {
+            // we always try at least one more time on timeout
+            assertThat(listener.isDone(), equalTo(false));
+            capturedRequests = transport.getCapturedRequestsAndClear();
+            assertThat(capturedRequests, arrayWithSize(1));
+            assertThat(capturedRequests[0].action, equalTo("testAction[p]"));
+            assertPhase(task, "waiting_on_primary");
+            transport.handleRemoteError(capturedRequests[0].requestId, randomRetryPrimaryException(shardId));
+            assertListenerThrows("must throw index not found exception", listener, ElasticsearchException.class);
+            assertPhase(task, "failed");
+        } else {
+            assertThat(listener.isDone(), equalTo(false));
+            // generate a CS change
+            setState(clusterService, clusterService.state());
+            capturedRequests = transport.getCapturedRequestsAndClear();
+            assertThat(capturedRequests, arrayWithSize(1));
+            assertThat(capturedRequests[0].action, equalTo("testAction[p]"));
+        }
+    }
+
+    private ElasticsearchException randomRetryPrimaryException(ShardId shardId) {
+        return randomFrom(
+            new ShardNotFoundException(shardId),
+            new IndexNotFoundException(shardId.getIndex()),
+            new IndexShardClosedException(shardId),
+            new EngineClosedException(shardId),
+            new TransportReplicationAction.RetryOnPrimaryException(shardId, "hello")
+        );
+    }
+
     public void testRoutePhaseExecutesRequest() {
     public void testRoutePhaseExecutesRequest() {
         final String index = "test";
         final String index = "test";
         final ShardId shardId = new ShardId(index, "_na_", 0);
         final ShardId shardId = new ShardId(index, "_na_", 0);
@@ -449,7 +506,7 @@ public class TransportReplicationActionTests extends ESTestCase {
         PlainActionFuture<Response> listener = new PlainActionFuture<>();
         PlainActionFuture<Response> listener = new PlainActionFuture<>();
         ReplicationTask task = maybeTask();
         ReplicationTask task = maybeTask();
         TransportReplicationAction<Request, Request, Response>.PrimaryPhase primaryPhase = actionWithRelocatingReplicasAfterPrimaryOp.new PrimaryPhase(
         TransportReplicationAction<Request, Request, Response>.PrimaryPhase primaryPhase = actionWithRelocatingReplicasAfterPrimaryOp.new PrimaryPhase(
-                task, request, createTransportChannel(listener));
+            task, request, createTransportChannel(listener));
         primaryPhase.run();
         primaryPhase.run();
         assertThat("request was not processed on primary", request.processedOnPrimary.get(), equalTo(true));
         assertThat("request was not processed on primary", request.processedOnPrimary.get(), equalTo(true));
         ShardRouting relocatingReplicaShard = stateWithRelocatingReplica.getRoutingTable().shardRoutingTable(index, shardId.id()).replicaShards().get(0);
         ShardRouting relocatingReplicaShard = stateWithRelocatingReplica.getRoutingTable().shardRoutingTable(index, shardId.id()).replicaShards().get(0);
@@ -485,7 +542,7 @@ public class TransportReplicationActionTests extends ESTestCase {
         PlainActionFuture<Response> listener = new PlainActionFuture<>();
         PlainActionFuture<Response> listener = new PlainActionFuture<>();
         ReplicationTask task = maybeTask();
         ReplicationTask task = maybeTask();
         TransportReplicationAction<Request, Request, Response>.PrimaryPhase primaryPhase = actionWithDeletedIndexAfterPrimaryOp.new PrimaryPhase(
         TransportReplicationAction<Request, Request, Response>.PrimaryPhase primaryPhase = actionWithDeletedIndexAfterPrimaryOp.new PrimaryPhase(
-                task, request, createTransportChannel(listener));
+            task, request, createTransportChannel(listener));
         primaryPhase.run();
         primaryPhase.run();
         assertThat("request was not processed on primary", request.processedOnPrimary.get(), equalTo(true));
         assertThat("request was not processed on primary", request.processedOnPrimary.get(), equalTo(true));
         assertThat("replication phase should be skipped if index gets deleted after primary operation", transport.capturedRequestsByTargetNode().size(), equalTo(0));
         assertThat("replication phase should be skipped if index gets deleted after primary operation", transport.capturedRequestsByTargetNode().size(), equalTo(0));
@@ -529,8 +586,8 @@ public class TransportReplicationActionTests extends ESTestCase {
 
 
         setState(clusterService, state(index, true, ShardRoutingState.STARTED, replicaStates));
         setState(clusterService, state(index, true, ShardRoutingState.STARTED, replicaStates));
         logger.debug("using consistency level of [{}], assigned shards [{}], total shards [{}]. expecting op to [{}]. using state: \n{}",
         logger.debug("using consistency level of [{}], assigned shards [{}], total shards [{}]. expecting op to [{}]. using state: \n{}",
-                request.consistencyLevel(), 1 + assignedReplicas, 1 + assignedReplicas + unassignedReplicas, passesWriteConsistency ? "succeed" : "retry",
-                clusterService.state().prettyPrint());
+            request.consistencyLevel(), 1 + assignedReplicas, 1 + assignedReplicas + unassignedReplicas, passesWriteConsistency ? "succeed" : "retry",
+            clusterService.state().prettyPrint());
 
 
         final IndexShardRoutingTable shardRoutingTable = clusterService.state().routingTable().index(index).shard(shardId.id());
         final IndexShardRoutingTable shardRoutingTable = clusterService.state().routingTable().index(index).shard(shardId.id());
         PlainActionFuture<Response> listener = new PlainActionFuture<>();
         PlainActionFuture<Response> listener = new PlainActionFuture<>();
@@ -646,7 +703,7 @@ public class TransportReplicationActionTests extends ESTestCase {
 
 
         TransportChannel channel = createTransportChannel(listener, error::set);
         TransportChannel channel = createTransportChannel(listener, error::set);
         TransportReplicationAction<Request, Request, Response>.ReplicationPhase replicationPhase =
         TransportReplicationAction<Request, Request, Response>.ReplicationPhase replicationPhase =
-                action.new ReplicationPhase(task, request, new Response(), request.shardId(), channel, reference);
+            action.new ReplicationPhase(task, request, new Response(), request.shardId(), channel, reference);
 
 
         assertThat(replicationPhase.totalShards(), equalTo(totalShards));
         assertThat(replicationPhase.totalShards(), equalTo(totalShards));
         assertThat(replicationPhase.pending(), equalTo(assignedReplicas));
         assertThat(replicationPhase.pending(), equalTo(assignedReplicas));
@@ -656,7 +713,7 @@ public class TransportReplicationActionTests extends ESTestCase {
 
 
         HashMap<String, Request> nodesSentTo = new HashMap<>();
         HashMap<String, Request> nodesSentTo = new HashMap<>();
         boolean executeOnReplica =
         boolean executeOnReplica =
-                action.shouldExecuteReplication(clusterService.state().getMetaData().index(shardId.getIndex()).getSettings());
+            action.shouldExecuteReplication(clusterService.state().getMetaData().index(shardId.getIndex()).getSettings());
         for (CapturingTransport.CapturedRequest capturedRequest : capturedRequests) {
         for (CapturingTransport.CapturedRequest capturedRequest : capturedRequests) {
             // no duplicate requests
             // no duplicate requests
             Request replicationRequest = (Request) capturedRequest.request;
             Request replicationRequest = (Request) capturedRequest.request;
@@ -819,7 +876,7 @@ public class TransportReplicationActionTests extends ESTestCase {
         final ShardId shardId = new ShardId(index, "_na_", 0);
         final ShardId shardId = new ShardId(index, "_na_", 0);
         // one replica to make sure replication is attempted
         // one replica to make sure replication is attempted
         setState(clusterService, state(index, true,
         setState(clusterService, state(index, true,
-                ShardRoutingState.STARTED, ShardRoutingState.STARTED));
+            ShardRoutingState.STARTED, ShardRoutingState.STARTED));
         ShardRouting primaryShard = clusterService.state().routingTable().shardRoutingTable(shardId).primaryShard();
         ShardRouting primaryShard = clusterService.state().routingTable().shardRoutingTable(shardId).primaryShard();
         indexShardRouting.set(primaryShard);
         indexShardRouting.set(primaryShard);
         logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint());
         logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint());
@@ -856,7 +913,7 @@ public class TransportReplicationActionTests extends ESTestCase {
     public void testReplicasCounter() throws Exception {
     public void testReplicasCounter() throws Exception {
         final ShardId shardId = new ShardId("test", "_na_", 0);
         final ShardId shardId = new ShardId("test", "_na_", 0);
         setState(clusterService, state(shardId.getIndexName(), true,
         setState(clusterService, state(shardId.getIndexName(), true,
-                ShardRoutingState.STARTED, ShardRoutingState.STARTED));
+            ShardRoutingState.STARTED, ShardRoutingState.STARTED));
         action = new ActionWithDelay(Settings.EMPTY, "testActionWithExceptions", transportService, clusterService, threadPool);
         action = new ActionWithDelay(Settings.EMPTY, "testActionWithExceptions", transportService, clusterService, threadPool);
         final Action.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler();
         final Action.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler();
         final ReplicationTask task = maybeTask();
         final ReplicationTask task = maybeTask();
@@ -895,7 +952,7 @@ public class TransportReplicationActionTests extends ESTestCase {
         final String index = "test";
         final String index = "test";
         final ShardId shardId = new ShardId(index, "_na_", 0);
         final ShardId shardId = new ShardId(index, "_na_", 0);
         setState(clusterService, state(index, true,
         setState(clusterService, state(index, true,
-                ShardRoutingState.STARTED, ShardRoutingState.STARTED));
+            ShardRoutingState.STARTED, ShardRoutingState.STARTED));
         logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint());
         logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint());
         Request request = new Request(shardId).timeout("100ms");
         Request request = new Request(shardId).timeout("100ms");
         PlainActionFuture<Response> listener = new PlainActionFuture<>();
         PlainActionFuture<Response> listener = new PlainActionFuture<>();
@@ -915,7 +972,7 @@ public class TransportReplicationActionTests extends ESTestCase {
         final ShardId shardId = new ShardId(index, "_na_", 0);
         final ShardId shardId = new ShardId(index, "_na_", 0);
         boolean localPrimary = true;
         boolean localPrimary = true;
         setState(clusterService, state(index, localPrimary,
         setState(clusterService, state(index, localPrimary,
-                ShardRoutingState.STARTED, ShardRoutingState.STARTED));
+            ShardRoutingState.STARTED, ShardRoutingState.STARTED));
         Action action = new Action(Settings.EMPTY, "testAction", transportService, clusterService, threadPool) {
         Action action = new Action(Settings.EMPTY, "testAction", transportService, clusterService, threadPool) {
             @Override
             @Override
             protected void resolveRequest(MetaData metaData, String concreteIndex, Request request) {
             protected void resolveRequest(MetaData metaData, String concreteIndex, Request request) {
@@ -967,7 +1024,7 @@ public class TransportReplicationActionTests extends ESTestCase {
         // publish a new cluster state
         // publish a new cluster state
         boolean localPrimaryOnRetry = randomBoolean();
         boolean localPrimaryOnRetry = randomBoolean();
         setState(clusterService, state(index, localPrimaryOnRetry,
         setState(clusterService, state(index, localPrimaryOnRetry,
-                ShardRoutingState.STARTED, ShardRoutingState.STARTED));
+            ShardRoutingState.STARTED, ShardRoutingState.STARTED));
         CapturingTransport.CapturedRequest[] primaryRetry = transport.getCapturedRequestsAndClear();
         CapturingTransport.CapturedRequest[] primaryRetry = transport.getCapturedRequestsAndClear();
 
 
         // the request should be retried
         // the request should be retried
@@ -1083,8 +1140,8 @@ public class TransportReplicationActionTests extends ESTestCase {
                ClusterService clusterService,
                ClusterService clusterService,
                ThreadPool threadPool) {
                ThreadPool threadPool) {
             super(settings, actionName, transportService, clusterService, null, threadPool,
             super(settings, actionName, transportService, clusterService, null, threadPool,
-                    new ShardStateAction(settings, clusterService, transportService, null, null, threadPool),
-                    new ActionFilters(new HashSet<ActionFilter>()), new IndexNameExpressionResolver(Settings.EMPTY), Request::new, Request::new, ThreadPool.Names.SAME);
+                new ShardStateAction(settings, clusterService, transportService, null, null, threadPool),
+                new ActionFilters(new HashSet<ActionFilter>()), new IndexNameExpressionResolver(Settings.EMPTY), Request::new, Request::new, ThreadPool.Names.SAME);
         }
         }
 
 
         @Override
         @Override