|  | @@ -26,6 +26,7 @@ import org.elasticsearch.action.support.ActiveShardCount;
 | 
	
		
			
				|  |  |  import org.elasticsearch.action.support.PlainActionFuture;
 | 
	
		
			
				|  |  |  import org.elasticsearch.client.transport.NoNodeAvailableException;
 | 
	
		
			
				|  |  |  import org.elasticsearch.cluster.ClusterState;
 | 
	
		
			
				|  |  | +import org.elasticsearch.cluster.ESAllocationTestCase;
 | 
	
		
			
				|  |  |  import org.elasticsearch.cluster.action.shard.ShardStateAction;
 | 
	
		
			
				|  |  |  import org.elasticsearch.cluster.block.ClusterBlock;
 | 
	
		
			
				|  |  |  import org.elasticsearch.cluster.block.ClusterBlockException;
 | 
	
	
		
			
				|  | @@ -36,6 +37,7 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
 | 
	
		
			
				|  |  |  import org.elasticsearch.cluster.metadata.MetaData;
 | 
	
		
			
				|  |  |  import org.elasticsearch.cluster.node.DiscoveryNodes;
 | 
	
		
			
				|  |  |  import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
 | 
	
		
			
				|  |  | +import org.elasticsearch.cluster.routing.RoutingNode;
 | 
	
		
			
				|  |  |  import org.elasticsearch.cluster.routing.ShardRouting;
 | 
	
		
			
				|  |  |  import org.elasticsearch.cluster.routing.ShardRoutingState;
 | 
	
		
			
				|  |  |  import org.elasticsearch.cluster.routing.TestShardRouting;
 | 
	
	
		
			
				|  | @@ -47,21 +49,25 @@ import org.elasticsearch.common.io.stream.StreamInput;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.io.stream.StreamOutput;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.lease.Releasable;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.settings.Settings;
 | 
	
		
			
				|  |  | +import org.elasticsearch.index.Index;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.IndexNotFoundException;
 | 
	
		
			
				|  |  | +import org.elasticsearch.index.IndexService;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.engine.EngineClosedException;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.shard.IndexShard;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.shard.IndexShardClosedException;
 | 
	
		
			
				|  |  | +import org.elasticsearch.index.shard.IndexShardState;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.shard.ShardId;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.shard.ShardNotFoundException;
 | 
	
		
			
				|  |  | +import org.elasticsearch.indices.IndicesService;
 | 
	
		
			
				|  |  |  import org.elasticsearch.node.NodeClosedException;
 | 
	
		
			
				|  |  |  import org.elasticsearch.rest.RestStatus;
 | 
	
		
			
				|  |  | -import org.elasticsearch.cluster.ESAllocationTestCase;
 | 
	
		
			
				|  |  |  import org.elasticsearch.test.ESTestCase;
 | 
	
		
			
				|  |  |  import org.elasticsearch.test.transport.CapturingTransport;
 | 
	
		
			
				|  |  |  import org.elasticsearch.threadpool.TestThreadPool;
 | 
	
		
			
				|  |  |  import org.elasticsearch.threadpool.ThreadPool;
 | 
	
		
			
				|  |  |  import org.elasticsearch.transport.TransportChannel;
 | 
	
		
			
				|  |  |  import org.elasticsearch.transport.TransportException;
 | 
	
		
			
				|  |  | +import org.elasticsearch.transport.TransportRequest;
 | 
	
		
			
				|  |  |  import org.elasticsearch.transport.TransportResponse;
 | 
	
		
			
				|  |  |  import org.elasticsearch.transport.TransportResponseOptions;
 | 
	
		
			
				|  |  |  import org.elasticsearch.transport.TransportService;
 | 
	
	
		
			
				|  | @@ -75,12 +81,12 @@ import java.io.IOException;
 | 
	
		
			
				|  |  |  import java.util.Arrays;
 | 
	
		
			
				|  |  |  import java.util.HashSet;
 | 
	
		
			
				|  |  |  import java.util.List;
 | 
	
		
			
				|  |  | +import java.util.Locale;
 | 
	
		
			
				|  |  |  import java.util.concurrent.ExecutionException;
 | 
	
		
			
				|  |  |  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.Consumer;
 | 
	
		
			
				|  |  |  import java.util.stream.Collectors;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.state;
 | 
	
	
		
			
				|  | @@ -93,12 +99,32 @@ import static org.hamcrest.Matchers.arrayWithSize;
 | 
	
		
			
				|  |  |  import static org.hamcrest.Matchers.equalTo;
 | 
	
		
			
				|  |  |  import static org.hamcrest.Matchers.instanceOf;
 | 
	
		
			
				|  |  |  import static org.hamcrest.Matchers.notNullValue;
 | 
	
		
			
				|  |  | +import static org.hamcrest.Matchers.nullValue;
 | 
	
		
			
				|  |  | +import static org.mockito.Matchers.any;
 | 
	
		
			
				|  |  | +import static org.mockito.Matchers.anyInt;
 | 
	
		
			
				|  |  | +import static org.mockito.Matchers.anyLong;
 | 
	
		
			
				|  |  | +import static org.mockito.Matchers.anyString;
 | 
	
		
			
				|  |  | +import static org.mockito.Mockito.doAnswer;
 | 
	
		
			
				|  |  | +import static org.mockito.Mockito.doThrow;
 | 
	
		
			
				|  |  |  import static org.mockito.Mockito.mock;
 | 
	
		
			
				|  |  |  import static org.mockito.Mockito.verify;
 | 
	
		
			
				|  |  |  import static org.mockito.Mockito.when;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  public class TransportReplicationActionTests extends ESTestCase {
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +    /**
 | 
	
		
			
				|  |  | +     * takes a request that was sent by a {@link TransportReplicationAction} and captured
 | 
	
		
			
				|  |  | +     * and returns the underlying request if it's wrapped or the original (cast to the expected type).
 | 
	
		
			
				|  |  | +     *
 | 
	
		
			
				|  |  | +     * This will throw a {@link ClassCastException} if the request is of the wrong type.
 | 
	
		
			
				|  |  | +     */
 | 
	
		
			
				|  |  | +    public static <R extends ReplicationRequest> R resolveRequest(TransportRequest requestOrWrappedRequest) {
 | 
	
		
			
				|  |  | +        if (requestOrWrappedRequest instanceof TransportReplicationAction.ConcreteShardRequest) {
 | 
	
		
			
				|  |  | +            requestOrWrappedRequest = ((TransportReplicationAction.ConcreteShardRequest)requestOrWrappedRequest).getRequest();
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +        return (R) requestOrWrappedRequest;
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |      private static ThreadPool threadPool;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      private ClusterService clusterService;
 | 
	
	
		
			
				|  | @@ -411,7 +437,7 @@ public class TransportReplicationActionTests extends ESTestCase {
 | 
	
		
			
				|  |  |              isRelocated.set(true);
 | 
	
		
			
				|  |  |              executeOnPrimary = false;
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  | -        action.new AsyncPrimaryAction(request, createTransportChannel(listener), task) {
 | 
	
		
			
				|  |  | +        action.new AsyncPrimaryAction(request, primaryShard.allocationId().getId(), createTransportChannel(listener), task) {
 | 
	
		
			
				|  |  |              @Override
 | 
	
		
			
				|  |  |              protected ReplicationOperation<Request, Request, Action.PrimaryResult> createReplicatedOperation(Request request,
 | 
	
		
			
				|  |  |                      ActionListener<Action.PrimaryResult> actionListener, Action.PrimaryShardReference primaryShardReference,
 | 
	
	
		
			
				|  | @@ -452,7 +478,8 @@ public class TransportReplicationActionTests extends ESTestCase {
 | 
	
		
			
				|  |  |          final String index = "test";
 | 
	
		
			
				|  |  |          final ShardId shardId = new ShardId(index, "_na_", 0);
 | 
	
		
			
				|  |  |          ClusterState state = state(index, true, ShardRoutingState.RELOCATING);
 | 
	
		
			
				|  |  | -        String primaryTargetNodeId = state.getRoutingTable().shardRoutingTable(shardId).primaryShard().relocatingNodeId();
 | 
	
		
			
				|  |  | +        final ShardRouting primaryShard = state.getRoutingTable().shardRoutingTable(shardId).primaryShard();
 | 
	
		
			
				|  |  | +        String primaryTargetNodeId = primaryShard.relocatingNodeId();
 | 
	
		
			
				|  |  |          // simulate execution of the primary phase on the relocation target node
 | 
	
		
			
				|  |  |          state = ClusterState.builder(state).nodes(DiscoveryNodes.builder(state.nodes()).localNodeId(primaryTargetNodeId)).build();
 | 
	
		
			
				|  |  |          setState(clusterService, state);
 | 
	
	
		
			
				|  | @@ -460,7 +487,7 @@ public class TransportReplicationActionTests extends ESTestCase {
 | 
	
		
			
				|  |  |          PlainActionFuture<Response> listener = new PlainActionFuture<>();
 | 
	
		
			
				|  |  |          ReplicationTask task = maybeTask();
 | 
	
		
			
				|  |  |          AtomicBoolean executed = new AtomicBoolean();
 | 
	
		
			
				|  |  | -        action.new AsyncPrimaryAction(request, createTransportChannel(listener), task) {
 | 
	
		
			
				|  |  | +        action.new AsyncPrimaryAction(request, primaryShard.allocationId().getRelocationId(), createTransportChannel(listener), task) {
 | 
	
		
			
				|  |  |              @Override
 | 
	
		
			
				|  |  |              protected ReplicationOperation<Request, Request, Action.PrimaryResult> createReplicatedOperation(Request request,
 | 
	
		
			
				|  |  |                      ActionListener<Action.PrimaryResult> actionListener, Action.PrimaryShardReference primaryShardReference,
 | 
	
	
		
			
				|  | @@ -473,6 +500,11 @@ public class TransportReplicationActionTests extends ESTestCase {
 | 
	
		
			
				|  |  |                      }
 | 
	
		
			
				|  |  |                  };
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +            @Override
 | 
	
		
			
				|  |  | +            public void onFailure(Exception e) {
 | 
	
		
			
				|  |  | +                throw new RuntimeException(e);
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  |          }.run();
 | 
	
		
			
				|  |  |          assertThat(executed.get(), equalTo(true));
 | 
	
		
			
				|  |  |          assertPhase(task, "finished");
 | 
	
	
		
			
				|  | @@ -596,7 +628,9 @@ public class TransportReplicationActionTests extends ESTestCase {
 | 
	
		
			
				|  |  |          state = ClusterState.builder(state).metaData(metaData).build();
 | 
	
		
			
				|  |  |          setState(clusterService, state);
 | 
	
		
			
				|  |  |          AtomicBoolean executed = new AtomicBoolean();
 | 
	
		
			
				|  |  | -        action.new AsyncPrimaryAction(new Request(shardId), createTransportChannel(new PlainActionFuture<>()), null) {
 | 
	
		
			
				|  |  | +        ShardRouting primaryShard = state.routingTable().shardRoutingTable(shardId).primaryShard();
 | 
	
		
			
				|  |  | +        action.new AsyncPrimaryAction(new Request(shardId), primaryShard.allocationId().getId(),
 | 
	
		
			
				|  |  | +            createTransportChannel(new PlainActionFuture<>()), null) {
 | 
	
		
			
				|  |  |              @Override
 | 
	
		
			
				|  |  |              protected ReplicationOperation<Request, Request, Action.PrimaryResult> createReplicatedOperation(Request request,
 | 
	
		
			
				|  |  |                      ActionListener<Action.PrimaryResult> actionListener, Action.PrimaryShardReference primaryShardReference,
 | 
	
	
		
			
				|  | @@ -613,8 +647,10 @@ public class TransportReplicationActionTests extends ESTestCase {
 | 
	
		
			
				|  |  |          final String index = "test";
 | 
	
		
			
				|  |  |          final ShardId shardId = new ShardId(index, "_na_", 0);
 | 
	
		
			
				|  |  |          // no replica, we only want to test on primary
 | 
	
		
			
				|  |  | -        setState(clusterService, state(index, true, ShardRoutingState.STARTED));
 | 
	
		
			
				|  |  | +        final ClusterState state = state(index, true, ShardRoutingState.STARTED);
 | 
	
		
			
				|  |  | +        setState(clusterService, state);
 | 
	
		
			
				|  |  |          logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint());
 | 
	
		
			
				|  |  | +        final ShardRouting primaryShard = state.routingTable().shardRoutingTable(shardId).primaryShard();
 | 
	
		
			
				|  |  |          Request request = new Request(shardId);
 | 
	
		
			
				|  |  |          PlainActionFuture<Response> listener = new PlainActionFuture<>();
 | 
	
		
			
				|  |  |          ReplicationTask task = maybeTask();
 | 
	
	
		
			
				|  | @@ -622,7 +658,7 @@ public class TransportReplicationActionTests extends ESTestCase {
 | 
	
		
			
				|  |  |          final boolean throwExceptionOnCreation = i == 1;
 | 
	
		
			
				|  |  |          final boolean throwExceptionOnRun = i == 2;
 | 
	
		
			
				|  |  |          final boolean respondWithError = i == 3;
 | 
	
		
			
				|  |  | -        action.new AsyncPrimaryAction(request, createTransportChannel(listener), task) {
 | 
	
		
			
				|  |  | +        action.new AsyncPrimaryAction(request, primaryShard.allocationId().getId(), createTransportChannel(listener), task) {
 | 
	
		
			
				|  |  |              @Override
 | 
	
		
			
				|  |  |              protected ReplicationOperation<Request, Request, Action.PrimaryResult> createReplicatedOperation(Request request,
 | 
	
		
			
				|  |  |                      ActionListener<Action.PrimaryResult> actionListener, Action.PrimaryShardReference primaryShardReference,
 | 
	
	
		
			
				|  | @@ -666,8 +702,9 @@ public class TransportReplicationActionTests extends ESTestCase {
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      public void testReplicasCounter() throws Exception {
 | 
	
		
			
				|  |  |          final ShardId shardId = new ShardId("test", "_na_", 0);
 | 
	
		
			
				|  |  | -        setState(clusterService, state(shardId.getIndexName(), true,
 | 
	
		
			
				|  |  | -            ShardRoutingState.STARTED, ShardRoutingState.STARTED));
 | 
	
		
			
				|  |  | +        final ClusterState state = state(shardId.getIndexName(), true, ShardRoutingState.STARTED, ShardRoutingState.STARTED);
 | 
	
		
			
				|  |  | +        setState(clusterService, state);
 | 
	
		
			
				|  |  | +        final ShardRouting replicaRouting = state.getRoutingTable().shardRoutingTable(shardId).replicaShards().get(0);
 | 
	
		
			
				|  |  |          boolean throwException = randomBoolean();
 | 
	
		
			
				|  |  |          final ReplicationTask task = maybeTask();
 | 
	
		
			
				|  |  |          Action action = new Action(Settings.EMPTY, "testActionWithExceptions", transportService, clusterService, threadPool) {
 | 
	
	
		
			
				|  | @@ -683,7 +720,8 @@ public class TransportReplicationActionTests extends ESTestCase {
 | 
	
		
			
				|  |  |          };
 | 
	
		
			
				|  |  |          final Action.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler();
 | 
	
		
			
				|  |  |          try {
 | 
	
		
			
				|  |  | -            replicaOperationTransportHandler.messageReceived(new Request().setShardId(shardId),
 | 
	
		
			
				|  |  | +            replicaOperationTransportHandler.messageReceived(
 | 
	
		
			
				|  |  | +                action.new ConcreteShardRequest(new Request().setShardId(shardId), replicaRouting.allocationId().getId()),
 | 
	
		
			
				|  |  |                  createTransportChannel(new PlainActionFuture<>()), task);
 | 
	
		
			
				|  |  |          } catch (ElasticsearchException e) {
 | 
	
		
			
				|  |  |              assertThat(e.getMessage(), containsString("simulated"));
 | 
	
	
		
			
				|  | @@ -725,6 +763,111 @@ public class TransportReplicationActionTests extends ESTestCase {
 | 
	
		
			
				|  |  |          assertEquals(ActiveShardCount.from(requestWaitForActiveShards), request.waitForActiveShards());
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +    /** test that a primary request is rejected if it arrives at a shard with a wrong allocation id */
 | 
	
		
			
				|  |  | +    public void testPrimaryActionRejectsWrongAid() throws Exception {
 | 
	
		
			
				|  |  | +        final String index = "test";
 | 
	
		
			
				|  |  | +        final ShardId shardId = new ShardId(index, "_na_", 0);
 | 
	
		
			
				|  |  | +        setState(clusterService, state(index, true, ShardRoutingState.STARTED));
 | 
	
		
			
				|  |  | +        PlainActionFuture<Response> listener = new PlainActionFuture<>();
 | 
	
		
			
				|  |  | +        Request request = new Request(shardId).timeout("1ms");
 | 
	
		
			
				|  |  | +            action.new PrimaryOperationTransportHandler().messageReceived(
 | 
	
		
			
				|  |  | +                action.new ConcreteShardRequest(request, "_not_a_valid_aid_"),
 | 
	
		
			
				|  |  | +                createTransportChannel(listener), maybeTask()
 | 
	
		
			
				|  |  | +            );
 | 
	
		
			
				|  |  | +        try {
 | 
	
		
			
				|  |  | +            listener.get();
 | 
	
		
			
				|  |  | +            fail("using a wrong aid didn't fail the operation");
 | 
	
		
			
				|  |  | +        } catch (ExecutionException execException) {
 | 
	
		
			
				|  |  | +            Throwable throwable = execException.getCause();
 | 
	
		
			
				|  |  | +            logger.debug("got exception:" , throwable);
 | 
	
		
			
				|  |  | +            assertTrue(throwable.getClass() + " is not a retry exception", action.retryPrimaryException(throwable));
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +    /** test that a replica request is rejected if it arrives at a shard with a wrong allocation id */
 | 
	
		
			
				|  |  | +    public void testReplicaActionRejectsWrongAid() throws Exception {
 | 
	
		
			
				|  |  | +        final String index = "test";
 | 
	
		
			
				|  |  | +        final ShardId shardId = new ShardId(index, "_na_", 0);
 | 
	
		
			
				|  |  | +        ClusterState state = state(index, false, ShardRoutingState.STARTED, ShardRoutingState.STARTED);
 | 
	
		
			
				|  |  | +        final ShardRouting replica = state.routingTable().shardRoutingTable(shardId).replicaShards().get(0);
 | 
	
		
			
				|  |  | +        // simulate execution of the node holding the replica
 | 
	
		
			
				|  |  | +        state = ClusterState.builder(state).nodes(DiscoveryNodes.builder(state.nodes()).localNodeId(replica.currentNodeId())).build();
 | 
	
		
			
				|  |  | +        setState(clusterService, state);
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        PlainActionFuture<Response> listener = new PlainActionFuture<>();
 | 
	
		
			
				|  |  | +        Request request = new Request(shardId).timeout("1ms");
 | 
	
		
			
				|  |  | +        action.new ReplicaOperationTransportHandler().messageReceived(
 | 
	
		
			
				|  |  | +            action.new ConcreteShardRequest(request, "_not_a_valid_aid_"),
 | 
	
		
			
				|  |  | +            createTransportChannel(listener), maybeTask()
 | 
	
		
			
				|  |  | +        );
 | 
	
		
			
				|  |  | +        try {
 | 
	
		
			
				|  |  | +            listener.get();
 | 
	
		
			
				|  |  | +            fail("using a wrong aid didn't fail the operation");
 | 
	
		
			
				|  |  | +        } catch (ExecutionException execException) {
 | 
	
		
			
				|  |  | +            Throwable throwable = execException.getCause();
 | 
	
		
			
				|  |  | +            if (action.retryPrimaryException(throwable) == false) {
 | 
	
		
			
				|  |  | +                throw new AssertionError("thrown exception is not retriable", throwable);
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  | +            assertThat(throwable.getMessage(), containsString("_not_a_valid_aid_"));
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +    /**
 | 
	
		
			
				|  |  | +     * test throwing a {@link org.elasticsearch.action.support.replication.TransportReplicationAction.RetryOnReplicaException}
 | 
	
		
			
				|  |  | +     * causes a retry
 | 
	
		
			
				|  |  | +     */
 | 
	
		
			
				|  |  | +    public void testRetryOnReplica() throws Exception {
 | 
	
		
			
				|  |  | +        final ShardId shardId = new ShardId("test", "_na_", 0);
 | 
	
		
			
				|  |  | +        ClusterState state = state(shardId.getIndexName(), true, ShardRoutingState.STARTED, ShardRoutingState.STARTED);
 | 
	
		
			
				|  |  | +        final ShardRouting replica = state.getRoutingTable().shardRoutingTable(shardId).replicaShards().get(0);
 | 
	
		
			
				|  |  | +        // simulate execution of the node holding the replica
 | 
	
		
			
				|  |  | +        state = ClusterState.builder(state).nodes(DiscoveryNodes.builder(state.nodes()).localNodeId(replica.currentNodeId())).build();
 | 
	
		
			
				|  |  | +        setState(clusterService, state);
 | 
	
		
			
				|  |  | +        AtomicBoolean throwException = new AtomicBoolean(true);
 | 
	
		
			
				|  |  | +        final ReplicationTask task = maybeTask();
 | 
	
		
			
				|  |  | +        Action action = new Action(Settings.EMPTY, "testActionWithExceptions", transportService, clusterService, threadPool) {
 | 
	
		
			
				|  |  | +            @Override
 | 
	
		
			
				|  |  | +            protected ReplicaResult shardOperationOnReplica(Request request) {
 | 
	
		
			
				|  |  | +                assertPhase(task, "replica");
 | 
	
		
			
				|  |  | +                if (throwException.get()) {
 | 
	
		
			
				|  |  | +                    throw new RetryOnReplicaException(shardId, "simulation");
 | 
	
		
			
				|  |  | +                }
 | 
	
		
			
				|  |  | +                return new ReplicaResult();
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  | +        };
 | 
	
		
			
				|  |  | +        final Action.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler();
 | 
	
		
			
				|  |  | +        final PlainActionFuture<Response> listener = new PlainActionFuture<>();
 | 
	
		
			
				|  |  | +        final Request request = new Request().setShardId(shardId);
 | 
	
		
			
				|  |  | +        request.primaryTerm(state.metaData().getIndexSafe(shardId.getIndex()).primaryTerm(shardId.id()));
 | 
	
		
			
				|  |  | +        replicaOperationTransportHandler.messageReceived(
 | 
	
		
			
				|  |  | +                action.new ConcreteShardRequest(request, replica.allocationId().getId()),
 | 
	
		
			
				|  |  | +                createTransportChannel(listener), task);
 | 
	
		
			
				|  |  | +        if (listener.isDone()) {
 | 
	
		
			
				|  |  | +            listener.get(); // fail with the exception if there
 | 
	
		
			
				|  |  | +            fail("listener shouldn't be done");
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        // no retry yet
 | 
	
		
			
				|  |  | +        List<CapturingTransport.CapturedRequest> capturedRequests =
 | 
	
		
			
				|  |  | +            transport.getCapturedRequestsByTargetNodeAndClear().get(replica.currentNodeId());
 | 
	
		
			
				|  |  | +        assertThat(capturedRequests, nullValue());
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        // release the waiting
 | 
	
		
			
				|  |  | +        throwException.set(false);
 | 
	
		
			
				|  |  | +        setState(clusterService, state);
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        capturedRequests = transport.getCapturedRequestsByTargetNodeAndClear().get(replica.currentNodeId());
 | 
	
		
			
				|  |  | +        assertThat(capturedRequests, notNullValue());
 | 
	
		
			
				|  |  | +        assertThat(capturedRequests.size(), equalTo(1));
 | 
	
		
			
				|  |  | +        final CapturingTransport.CapturedRequest capturedRequest = capturedRequests.get(0);
 | 
	
		
			
				|  |  | +        assertThat(capturedRequest.action, equalTo("testActionWithExceptions[r]"));
 | 
	
		
			
				|  |  | +        assertThat(capturedRequest.request, instanceOf(TransportReplicationAction.ConcreteShardRequest.class));
 | 
	
		
			
				|  |  | +        assertThat(((TransportReplicationAction.ConcreteShardRequest) capturedRequest.request).getRequest(), equalTo(request));
 | 
	
		
			
				|  |  | +        assertThat(((TransportReplicationAction.ConcreteShardRequest) capturedRequest.request).getTargetAllocationID(),
 | 
	
		
			
				|  |  | +            equalTo(replica.allocationId().getId()));
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |      private void assertIndexShardCounter(int expected) {
 | 
	
		
			
				|  |  |          assertThat(count.get(), equalTo(expected));
 | 
	
		
			
				|  |  |      }
 | 
	
	
		
			
				|  | @@ -797,7 +940,7 @@ public class TransportReplicationActionTests extends ESTestCase {
 | 
	
		
			
				|  |  |          Action(Settings settings, String actionName, TransportService transportService,
 | 
	
		
			
				|  |  |                 ClusterService clusterService,
 | 
	
		
			
				|  |  |                 ThreadPool threadPool) {
 | 
	
		
			
				|  |  | -            super(settings, actionName, transportService, clusterService, null, threadPool,
 | 
	
		
			
				|  |  | +            super(settings, actionName, transportService, clusterService, mockIndicesService(clusterService), threadPool,
 | 
	
		
			
				|  |  |                  new ShardStateAction(settings, clusterService, transportService, null, null, threadPool),
 | 
	
		
			
				|  |  |                  new ActionFilters(new HashSet<>()), new IndexNameExpressionResolver(Settings.EMPTY),
 | 
	
		
			
				|  |  |                  Request::new, Request::new, ThreadPool.Names.SAME);
 | 
	
	
		
			
				|  | @@ -825,43 +968,76 @@ public class TransportReplicationActionTests extends ESTestCase {
 | 
	
		
			
				|  |  |          protected boolean resolveIndex() {
 | 
	
		
			
				|  |  |              return false;
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        @Override
 | 
	
		
			
				|  |  | -        protected void acquirePrimaryShardReference(ShardId shardId, ActionListener<PrimaryShardReference> onReferenceAcquired) {
 | 
	
		
			
				|  |  | -            count.incrementAndGet();
 | 
	
		
			
				|  |  | -            PrimaryShardReference primaryShardReference = new PrimaryShardReference(null, null) {
 | 
	
		
			
				|  |  | -                @Override
 | 
	
		
			
				|  |  | -                public boolean isRelocated() {
 | 
	
		
			
				|  |  | -                    return isRelocated.get();
 | 
	
		
			
				|  |  | -                }
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -                @Override
 | 
	
		
			
				|  |  | -                public void failShard(String reason, @Nullable Exception e) {
 | 
	
		
			
				|  |  | -                    throw new UnsupportedOperationException();
 | 
	
		
			
				|  |  | -                }
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -                @Override
 | 
	
		
			
				|  |  | -                public ShardRouting routingEntry() {
 | 
	
		
			
				|  |  | -                    ShardRouting shardRouting = clusterService.state().getRoutingTable()
 | 
	
		
			
				|  |  | -                        .shardRoutingTable(shardId).primaryShard();
 | 
	
		
			
				|  |  | -                    assert shardRouting != null;
 | 
	
		
			
				|  |  | -                    return shardRouting;
 | 
	
		
			
				|  |  | -                }
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -                @Override
 | 
	
		
			
				|  |  | -                public void close() {
 | 
	
		
			
				|  |  | -                    count.decrementAndGet();
 | 
	
		
			
				|  |  | -                }
 | 
	
		
			
				|  |  | -            };
 | 
	
		
			
				|  |  | +    final IndicesService mockIndicesService(ClusterService clusterService) {
 | 
	
		
			
				|  |  | +        final IndicesService indicesService = mock(IndicesService.class);
 | 
	
		
			
				|  |  | +        when(indicesService.indexServiceSafe(any(Index.class))).then(invocation -> {
 | 
	
		
			
				|  |  | +            Index index = (Index)invocation.getArguments()[0];
 | 
	
		
			
				|  |  | +            final ClusterState state = clusterService.state();
 | 
	
		
			
				|  |  | +            final IndexMetaData indexSafe = state.metaData().getIndexSafe(index);
 | 
	
		
			
				|  |  | +            return mockIndexService(indexSafe, clusterService);
 | 
	
		
			
				|  |  | +        });
 | 
	
		
			
				|  |  | +        when(indicesService.indexService(any(Index.class))).then(invocation -> {
 | 
	
		
			
				|  |  | +            Index index = (Index) invocation.getArguments()[0];
 | 
	
		
			
				|  |  | +            final ClusterState state = clusterService.state();
 | 
	
		
			
				|  |  | +            if (state.metaData().hasIndex(index.getName())) {
 | 
	
		
			
				|  |  | +                final IndexMetaData indexSafe = state.metaData().getIndexSafe(index);
 | 
	
		
			
				|  |  | +                return mockIndexService(clusterService.state().metaData().getIndexSafe(index), clusterService);
 | 
	
		
			
				|  |  | +            } else {
 | 
	
		
			
				|  |  | +                return null;
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  | +        });
 | 
	
		
			
				|  |  | +        return indicesService;
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -            onReferenceAcquired.onResponse(primaryShardReference);
 | 
	
		
			
				|  |  | -        }
 | 
	
		
			
				|  |  | +    final IndexService mockIndexService(final IndexMetaData indexMetaData, ClusterService clusterService) {
 | 
	
		
			
				|  |  | +        final IndexService indexService = mock(IndexService.class);
 | 
	
		
			
				|  |  | +        when(indexService.getShard(anyInt())).then(invocation -> {
 | 
	
		
			
				|  |  | +            int shard = (Integer) invocation.getArguments()[0];
 | 
	
		
			
				|  |  | +            final ShardId shardId = new ShardId(indexMetaData.getIndex(), shard);
 | 
	
		
			
				|  |  | +            if (shard > indexMetaData.getNumberOfShards()) {
 | 
	
		
			
				|  |  | +                throw new ShardNotFoundException(shardId);
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  | +            return mockIndexShard(shardId, clusterService);
 | 
	
		
			
				|  |  | +        });
 | 
	
		
			
				|  |  | +        return indexService;
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        @Override
 | 
	
		
			
				|  |  | -        protected void acquireReplicaOperationLock(ShardId shardId, long primaryTerm, ActionListener<Releasable> onLockAcquired) {
 | 
	
		
			
				|  |  | +    private IndexShard mockIndexShard(ShardId shardId, ClusterService clusterService) {
 | 
	
		
			
				|  |  | +        final IndexShard indexShard = mock(IndexShard.class);
 | 
	
		
			
				|  |  | +        doAnswer(invocation -> {
 | 
	
		
			
				|  |  | +            ActionListener<Releasable> callback = (ActionListener<Releasable>) invocation.getArguments()[0];
 | 
	
		
			
				|  |  |              count.incrementAndGet();
 | 
	
		
			
				|  |  | -            onLockAcquired.onResponse(count::decrementAndGet);
 | 
	
		
			
				|  |  | -        }
 | 
	
		
			
				|  |  | +            callback.onResponse(count::decrementAndGet);
 | 
	
		
			
				|  |  | +            return null;
 | 
	
		
			
				|  |  | +        }).when(indexShard).acquirePrimaryOperationLock(any(ActionListener.class), anyString());
 | 
	
		
			
				|  |  | +        doAnswer(invocation -> {
 | 
	
		
			
				|  |  | +            long term = (Long)invocation.getArguments()[0];
 | 
	
		
			
				|  |  | +            ActionListener<Releasable> callback = (ActionListener<Releasable>) invocation.getArguments()[1];
 | 
	
		
			
				|  |  | +            final long primaryTerm = indexShard.getPrimaryTerm();
 | 
	
		
			
				|  |  | +            if (term < primaryTerm) {
 | 
	
		
			
				|  |  | +                throw new IllegalArgumentException(String.format(Locale.ROOT, "%s operation term [%d] is too old (current [%d])",
 | 
	
		
			
				|  |  | +                    shardId, term, primaryTerm));
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  | +            count.incrementAndGet();
 | 
	
		
			
				|  |  | +            callback.onResponse(count::decrementAndGet);
 | 
	
		
			
				|  |  | +            return null;
 | 
	
		
			
				|  |  | +        }).when(indexShard).acquireReplicaOperationLock(anyLong(), any(ActionListener.class), anyString());
 | 
	
		
			
				|  |  | +        when(indexShard.routingEntry()).thenAnswer(invocationOnMock -> {
 | 
	
		
			
				|  |  | +            final ClusterState state = clusterService.state();
 | 
	
		
			
				|  |  | +            final RoutingNode node = state.getRoutingNodes().node(state.nodes().getLocalNodeId());
 | 
	
		
			
				|  |  | +            final ShardRouting routing = node.getByShardId(shardId);
 | 
	
		
			
				|  |  | +            if (routing == null) {
 | 
	
		
			
				|  |  | +                throw new ShardNotFoundException(shardId, "shard is no longer assigned to current node");
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  | +            return routing;
 | 
	
		
			
				|  |  | +        });
 | 
	
		
			
				|  |  | +        when(indexShard.state()).thenAnswer(invocationOnMock -> isRelocated.get() ? IndexShardState.RELOCATED : IndexShardState.STARTED);
 | 
	
		
			
				|  |  | +        doThrow(new AssertionError("failed shard is not supported")).when(indexShard).failShard(anyString(), any(Exception.class));
 | 
	
		
			
				|  |  | +        when(indexShard.getPrimaryTerm()).thenAnswer(i ->
 | 
	
		
			
				|  |  | +            clusterService.state().metaData().getIndexSafe(shardId.getIndex()).primaryTerm(shardId.id()));
 | 
	
		
			
				|  |  | +        return indexShard;
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      class NoopReplicationOperation extends ReplicationOperation<Request, Request, Action.PrimaryResult> {
 | 
	
	
		
			
				|  | @@ -879,11 +1055,6 @@ public class TransportReplicationActionTests extends ESTestCase {
 | 
	
		
			
				|  |  |       * Transport channel that is needed for replica operation testing.
 | 
	
		
			
				|  |  |       */
 | 
	
		
			
				|  |  |      public TransportChannel createTransportChannel(final PlainActionFuture<Response> listener) {
 | 
	
		
			
				|  |  | -        return createTransportChannel(listener, error -> {
 | 
	
		
			
				|  |  | -        });
 | 
	
		
			
				|  |  | -    }
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -    public TransportChannel createTransportChannel(final PlainActionFuture<Response> listener, Consumer<Throwable> consumer) {
 | 
	
		
			
				|  |  |          return new TransportChannel() {
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |              @Override
 | 
	
	
		
			
				|  | @@ -908,7 +1079,6 @@ public class TransportReplicationActionTests extends ESTestCase {
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |              @Override
 | 
	
		
			
				|  |  |              public void sendResponse(Exception exception) throws IOException {
 | 
	
		
			
				|  |  | -                consumer.accept(exception);
 | 
	
		
			
				|  |  |                  listener.onFailure(exception);
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  |  
 |