|
@@ -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);
|
|
|
}
|
|
|
|