|
@@ -135,7 +135,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
private ClusterService clusterService;
|
|
|
private TransportService transportService;
|
|
|
private CapturingTransport transport;
|
|
|
- private Action action;
|
|
|
+ private TestAction action;
|
|
|
private ShardStateAction shardStateAction;
|
|
|
|
|
|
/* *
|
|
@@ -159,7 +159,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
transportService.start();
|
|
|
transportService.acceptIncomingRequests();
|
|
|
shardStateAction = new ShardStateAction(Settings.EMPTY, clusterService, transportService, null, null, threadPool);
|
|
|
- action = new Action(Settings.EMPTY, "testAction", transportService, clusterService, shardStateAction, threadPool);
|
|
|
+ action = new TestAction(Settings.EMPTY, "testAction", transportService, clusterService, shardStateAction, threadPool);
|
|
|
}
|
|
|
|
|
|
@After
|
|
@@ -185,9 +185,10 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
|
|
|
public void testBlocks() throws ExecutionException, InterruptedException {
|
|
|
Request request = new Request();
|
|
|
- PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
|
|
+ PlainActionFuture<TestResponse> listener = new PlainActionFuture<>();
|
|
|
ReplicationTask task = maybeTask();
|
|
|
- Action action = new Action(Settings.EMPTY, "testActionWithBlocks", transportService, clusterService, shardStateAction, threadPool) {
|
|
|
+ TestAction action = new TestAction(Settings.EMPTY, "testActionWithBlocks",
|
|
|
+ transportService, clusterService, shardStateAction, threadPool) {
|
|
|
@Override
|
|
|
protected ClusterBlockLevel globalBlockLevel() {
|
|
|
return ClusterBlockLevel.WRITE;
|
|
@@ -197,7 +198,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
ClusterBlocks.Builder block = ClusterBlocks.builder()
|
|
|
.addGlobalBlock(new ClusterBlock(1, "non retryable", false, true, RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL));
|
|
|
setState(clusterService, ClusterState.builder(clusterService.state()).blocks(block));
|
|
|
- Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
|
|
|
+ TestAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
|
|
|
reroutePhase.run();
|
|
|
assertListenerThrows("primary phase should fail operation", listener, ClusterBlockException.class);
|
|
|
assertPhase(task, "failed");
|
|
@@ -226,7 +227,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
ClusterBlockException.class);
|
|
|
assertIndexShardUninitialized();
|
|
|
|
|
|
- action = new Action(Settings.EMPTY, "testActionWithNoBlocks", transportService, clusterService, shardStateAction, threadPool) {
|
|
|
+ action = new TestAction(Settings.EMPTY, "testActionWithNoBlocks", transportService, clusterService, shardStateAction, threadPool) {
|
|
|
@Override
|
|
|
protected ClusterBlockLevel globalBlockLevel() {
|
|
|
return null;
|
|
@@ -253,8 +254,8 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
logger.debug("--> using initial state:\n{}", clusterService.state());
|
|
|
|
|
|
Request request = new Request(shardId).timeout("1ms");
|
|
|
- PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
|
|
- Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
|
|
|
+ PlainActionFuture<TestResponse> listener = new PlainActionFuture<>();
|
|
|
+ TestAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
|
|
|
reroutePhase.run();
|
|
|
assertListenerThrows("unassigned primary didn't cause a timeout", listener, UnavailableShardsException.class);
|
|
|
assertPhase(task, "failed");
|
|
@@ -301,8 +302,8 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
logger.debug("--> relocation ongoing state:\n{}", clusterService.state());
|
|
|
|
|
|
Request request = new Request(shardId).timeout("1ms").routedBasedOnClusterVersion(clusterService.state().version() + 1);
|
|
|
- PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
|
|
- Action.ReroutePhase reroutePhase = action.new ReroutePhase(null, request, listener);
|
|
|
+ PlainActionFuture<TestResponse> listener = new PlainActionFuture<>();
|
|
|
+ TestAction.ReroutePhase reroutePhase = action.new ReroutePhase(null, request, listener);
|
|
|
reroutePhase.run();
|
|
|
assertListenerThrows("cluster state too old didn't cause a timeout", listener, UnavailableShardsException.class);
|
|
|
assertTrue(request.isRetrySet.compareAndSet(true, false));
|
|
@@ -340,10 +341,10 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED));
|
|
|
logger.debug("--> using initial state:\n{}", clusterService.state());
|
|
|
Request request = new Request(new ShardId("unknown_index", "_na_", 0)).timeout("1ms");
|
|
|
- PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
|
|
+ PlainActionFuture<TestResponse> listener = new PlainActionFuture<>();
|
|
|
ReplicationTask task = maybeTask();
|
|
|
|
|
|
- Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
|
|
|
+ TestAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
|
|
|
reroutePhase.run();
|
|
|
assertListenerThrows("must throw index not found exception", listener, IndexNotFoundException.class);
|
|
|
assertPhase(task, "failed");
|
|
@@ -364,17 +365,18 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
ShardRoutingState.UNASSIGNED), new CloseIndexRequest(index)));
|
|
|
logger.debug("--> using initial state:\n{}", clusterService.state());
|
|
|
Request request = new Request(new ShardId("test", "_na_", 0)).timeout("1ms");
|
|
|
- PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
|
|
+ PlainActionFuture<TestResponse> listener = new PlainActionFuture<>();
|
|
|
ReplicationTask task = maybeTask();
|
|
|
|
|
|
ClusterBlockLevel indexBlockLevel = randomBoolean() ? ClusterBlockLevel.WRITE : null;
|
|
|
- Action action = new Action(Settings.EMPTY, "testActionWithBlocks", transportService, clusterService, shardStateAction, threadPool) {
|
|
|
+ TestAction action = new TestAction(Settings.EMPTY, "testActionWithBlocks", transportService,
|
|
|
+ clusterService, shardStateAction, threadPool) {
|
|
|
@Override
|
|
|
protected ClusterBlockLevel indexBlockLevel() {
|
|
|
return indexBlockLevel;
|
|
|
}
|
|
|
};
|
|
|
- Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
|
|
|
+ TestAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
|
|
|
reroutePhase.run();
|
|
|
if (indexBlockLevel == ClusterBlockLevel.WRITE) {
|
|
|
assertListenerThrows("must throw block exception", listener, ClusterBlockException.class);
|
|
@@ -398,10 +400,10 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
} else {
|
|
|
request.timeout("1h");
|
|
|
}
|
|
|
- PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
|
|
+ PlainActionFuture<TestResponse> listener = new PlainActionFuture<>();
|
|
|
ReplicationTask task = maybeTask();
|
|
|
|
|
|
- Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
|
|
|
+ TestAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
|
|
|
reroutePhase.run();
|
|
|
CapturingTransport.CapturedRequest[] capturedRequests = transport.getCapturedRequestsAndClear();
|
|
|
assertThat(capturedRequests, arrayWithSize(1));
|
|
@@ -452,9 +454,9 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
final IndexShardRoutingTable shardRoutingTable = clusterService.state().routingTable().index(index).shard(shardId.id());
|
|
|
final String primaryNodeId = shardRoutingTable.primaryShard().currentNodeId();
|
|
|
Request request = new Request(shardId);
|
|
|
- PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
|
|
+ PlainActionFuture<TestResponse> listener = new PlainActionFuture<>();
|
|
|
|
|
|
- Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
|
|
|
+ TestAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
|
|
|
reroutePhase.run();
|
|
|
assertThat(request.shardId(), equalTo(shardId));
|
|
|
logger.info("--> primary is assigned to [{}], checking request forwarded", primaryNodeId);
|
|
@@ -479,7 +481,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
ClusterState state = stateWithActivePrimary(index, true, randomInt(5));
|
|
|
setState(clusterService, state);
|
|
|
Request request = new Request(shardId).timeout("1ms");
|
|
|
- PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
|
|
+ PlainActionFuture<TestResponse> listener = new PlainActionFuture<>();
|
|
|
ReplicationTask task = maybeTask();
|
|
|
AtomicBoolean executed = new AtomicBoolean();
|
|
|
|
|
@@ -492,11 +494,12 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
}
|
|
|
action.new AsyncPrimaryAction(request, primaryShard.allocationId().getId(), createTransportChannel(listener), task) {
|
|
|
@Override
|
|
|
- protected ReplicationOperation<Request, Request, TransportReplicationAction.PrimaryResult<Request, Response>>
|
|
|
- createReplicatedOperation(Request request,
|
|
|
- ActionListener<TransportReplicationAction.PrimaryResult<Request, Response>> actionListener,
|
|
|
- TransportReplicationAction<Request, Request, Response>.PrimaryShardReference primaryShardReference,
|
|
|
- boolean executeOnReplicas) {
|
|
|
+ protected ReplicationOperation<Request, Request, TransportReplicationAction.PrimaryResult<Request, TestResponse>>
|
|
|
+ createReplicatedOperation(
|
|
|
+ Request request,
|
|
|
+ ActionListener<TransportReplicationAction.PrimaryResult<Request, TestResponse>> actionListener,
|
|
|
+ TransportReplicationAction<Request, Request, TestResponse>.PrimaryShardReference primaryShardReference,
|
|
|
+ boolean executeOnReplicas) {
|
|
|
return new NoopReplicationOperation(request, actionListener) {
|
|
|
public void execute() throws Exception {
|
|
|
assertPhase(task, "primary");
|
|
@@ -521,7 +524,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
assertThat(requests.size(), equalTo(1));
|
|
|
assertThat("primary request was not delegated to relocation target", requests.get(0).action, equalTo("testAction[p]"));
|
|
|
assertPhase(task, "primary_delegation");
|
|
|
- transport.handleResponse(requests.get(0).requestId, new Response());
|
|
|
+ transport.handleResponse(requests.get(0).requestId, new TestResponse());
|
|
|
assertTrue(listener.isDone());
|
|
|
listener.get();
|
|
|
assertPhase(task, "finished");
|
|
@@ -539,16 +542,17 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
state = ClusterState.builder(state).nodes(DiscoveryNodes.builder(state.nodes()).localNodeId(primaryTargetNodeId)).build();
|
|
|
setState(clusterService, state);
|
|
|
Request request = new Request(shardId).timeout("1ms");
|
|
|
- PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
|
|
+ PlainActionFuture<TestResponse> listener = new PlainActionFuture<>();
|
|
|
ReplicationTask task = maybeTask();
|
|
|
AtomicBoolean executed = new AtomicBoolean();
|
|
|
action.new AsyncPrimaryAction(request, primaryShard.allocationId().getRelocationId(), createTransportChannel(listener), task) {
|
|
|
@Override
|
|
|
- protected ReplicationOperation<Request, Request, TransportReplicationAction.PrimaryResult<Request, Response>>
|
|
|
- createReplicatedOperation(Request request,
|
|
|
- ActionListener<TransportReplicationAction.PrimaryResult<Request, Response>> actionListener,
|
|
|
- TransportReplicationAction<Request, Request, Response>.PrimaryShardReference primaryShardReference,
|
|
|
- boolean executeOnReplicas) {
|
|
|
+ protected ReplicationOperation<Request, Request, TransportReplicationAction.PrimaryResult<Request, TestResponse>>
|
|
|
+ createReplicatedOperation(
|
|
|
+ Request request,
|
|
|
+ ActionListener<TransportReplicationAction.PrimaryResult<Request, TestResponse>> actionListener,
|
|
|
+ TransportReplicationAction<Request, Request, TestResponse>.PrimaryShardReference primaryShardReference,
|
|
|
+ boolean executeOnReplicas) {
|
|
|
return new NoopReplicationOperation(request, actionListener) {
|
|
|
public void execute() throws Exception {
|
|
|
assertPhase(task, "primary");
|
|
@@ -579,7 +583,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
fail("releasable is closed twice");
|
|
|
}
|
|
|
};
|
|
|
- Action.PrimaryShardReference primary = action.new PrimaryShardReference(shard, releasable);
|
|
|
+ TestAction.PrimaryShardReference primary = action.new PrimaryShardReference(shard, releasable);
|
|
|
final Request request = new Request();
|
|
|
Request replicaRequest = (Request) primary.perform(request).replicaRequest;
|
|
|
|
|
@@ -596,7 +600,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
}
|
|
|
|
|
|
public void testReplicaProxy() throws InterruptedException, ExecutionException {
|
|
|
- Action.ReplicasProxy proxy = action.new ReplicasProxy();
|
|
|
+ ReplicationOperation.Replicas proxy = action.newReplicasProxy();
|
|
|
final String index = "test";
|
|
|
final ShardId shardId = new ShardId(index, "_na_", 0);
|
|
|
ClusterState state = stateWithActivePrimary(index, true, 1 + randomInt(3), randomInt(2));
|
|
@@ -636,43 +640,15 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
assertListenerThrows("listener should reflect remote error", listener, TransportException.class);
|
|
|
}
|
|
|
|
|
|
- AtomicReference<Throwable> failure = new AtomicReference<>();
|
|
|
- AtomicReference<Throwable> ignoredFailure = new AtomicReference<>();
|
|
|
+ AtomicReference<Object> failure = new AtomicReference<>();
|
|
|
+ AtomicReference<Object> ignoredFailure = new AtomicReference<>();
|
|
|
AtomicBoolean success = new AtomicBoolean();
|
|
|
- proxy.failShard(replica, randomIntBetween(1, 10), "test", new ElasticsearchException("simulated"),
|
|
|
- () -> success.set(true), failure::set, ignoredFailure::set
|
|
|
+ proxy.failShardIfNeeded(replica, randomIntBetween(1, 10), "test", new ElasticsearchException("simulated"),
|
|
|
+ () -> success.set(true), failure::set, ignoredFailure::set
|
|
|
);
|
|
|
CapturingTransport.CapturedRequest[] shardFailedRequests = transport.getCapturedRequestsAndClear();
|
|
|
- assertEquals(1, shardFailedRequests.length);
|
|
|
- CapturingTransport.CapturedRequest shardFailedRequest = shardFailedRequests[0];
|
|
|
- ShardStateAction.ShardEntry shardEntry = (ShardStateAction.ShardEntry) shardFailedRequest.request;
|
|
|
- // the shard the request was sent to and the shard to be failed should be the same
|
|
|
- assertEquals(shardEntry.getShardId(), replica.shardId());
|
|
|
- assertEquals(shardEntry.getAllocationId(), replica.allocationId().getId());
|
|
|
- if (randomBoolean()) {
|
|
|
- // simulate success
|
|
|
- transport.handleResponse(shardFailedRequest.requestId, TransportResponse.Empty.INSTANCE);
|
|
|
- assertTrue(success.get());
|
|
|
- assertNull(failure.get());
|
|
|
- assertNull(ignoredFailure.get());
|
|
|
-
|
|
|
- } else if (randomBoolean()) {
|
|
|
- // simulate the primary has been demoted
|
|
|
- transport.handleRemoteError(shardFailedRequest.requestId,
|
|
|
- new ShardStateAction.NoLongerPrimaryShardException(replica.shardId(),
|
|
|
- "shard-failed-test"));
|
|
|
- assertFalse(success.get());
|
|
|
- assertNotNull(failure.get());
|
|
|
- assertNull(ignoredFailure.get());
|
|
|
-
|
|
|
- } else {
|
|
|
- // simulated an "ignored" exception
|
|
|
- transport.handleRemoteError(shardFailedRequest.requestId,
|
|
|
- new NodeClosedException(state.nodes().getLocalNode()));
|
|
|
- assertFalse(success.get());
|
|
|
- assertNull(failure.get());
|
|
|
- assertNotNull(ignoredFailure.get());
|
|
|
- }
|
|
|
+ // A replication action doesn't not fail the request
|
|
|
+ assertEquals(0, shardFailedRequests.length);
|
|
|
}
|
|
|
|
|
|
public void testShadowIndexDisablesReplication() throws Exception {
|
|
@@ -691,9 +667,9 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
action.new AsyncPrimaryAction(new Request(shardId), primaryShard.allocationId().getId(),
|
|
|
createTransportChannel(new PlainActionFuture<>()), null) {
|
|
|
@Override
|
|
|
- protected ReplicationOperation<Request, Request, Action.PrimaryResult<Request, Response>> createReplicatedOperation(
|
|
|
- Request request, ActionListener<TransportReplicationAction.PrimaryResult<Request, Response>> actionListener,
|
|
|
- TransportReplicationAction<Request, Request, Response>.PrimaryShardReference primaryShardReference,
|
|
|
+ protected ReplicationOperation<Request, Request, TestAction.PrimaryResult<Request, TestResponse>> createReplicatedOperation(
|
|
|
+ Request request, ActionListener<TransportReplicationAction.PrimaryResult<Request, TestResponse>> actionListener,
|
|
|
+ TransportReplicationAction<Request, Request, TestResponse>.PrimaryShardReference primaryShardReference,
|
|
|
boolean executeOnReplicas) {
|
|
|
assertFalse(executeOnReplicas);
|
|
|
assertFalse(executed.getAndSet(true));
|
|
@@ -715,7 +691,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
Request request = new Request(shardId);
|
|
|
TransportReplicationAction.ConcreteShardRequest<Request> concreteShardRequest =
|
|
|
new TransportReplicationAction.ConcreteShardRequest<>(request, routingEntry.allocationId().getId());
|
|
|
- PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
|
|
+ PlainActionFuture<TestResponse> listener = new PlainActionFuture<>();
|
|
|
|
|
|
|
|
|
final IndexShard shard = mock(IndexShard.class);
|
|
@@ -730,10 +706,10 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
}
|
|
|
};
|
|
|
|
|
|
- Action action =
|
|
|
- new Action(Settings.EMPTY, "testSeqNoIsSetOnPrimary", transportService, clusterService, shardStateAction, threadPool);
|
|
|
+ TestAction action =
|
|
|
+ new TestAction(Settings.EMPTY, "testSeqNoIsSetOnPrimary", transportService, clusterService, shardStateAction, threadPool);
|
|
|
|
|
|
- TransportReplicationAction<Request, Request, Response>.PrimaryOperationTransportHandler primaryPhase =
|
|
|
+ TransportReplicationAction<Request, Request, TestResponse>.PrimaryOperationTransportHandler primaryPhase =
|
|
|
action.new PrimaryOperationTransportHandler();
|
|
|
primaryPhase.messageReceived(concreteShardRequest, createTransportChannel(listener), null);
|
|
|
CapturingTransport.CapturedRequest[] requestsToReplicas = transport.capturedRequests();
|
|
@@ -751,7 +727,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
logger.debug("--> using initial state:\n{}", clusterService.state());
|
|
|
final ShardRouting primaryShard = state.routingTable().shardRoutingTable(shardId).primaryShard();
|
|
|
Request request = new Request(shardId);
|
|
|
- PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
|
|
+ PlainActionFuture<TestResponse> listener = new PlainActionFuture<>();
|
|
|
ReplicationTask task = maybeTask();
|
|
|
int i = randomInt(3);
|
|
|
final boolean throwExceptionOnCreation = i == 1;
|
|
@@ -759,11 +735,12 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
final boolean respondWithError = i == 3;
|
|
|
action.new AsyncPrimaryAction(request, primaryShard.allocationId().getId(), createTransportChannel(listener), task) {
|
|
|
@Override
|
|
|
- protected ReplicationOperation<Request, Request, TransportReplicationAction.PrimaryResult<Request, Response>>
|
|
|
- createReplicatedOperation(Request request,
|
|
|
- ActionListener<TransportReplicationAction.PrimaryResult<Request, Response>> actionListener,
|
|
|
- TransportReplicationAction<Request, Request, Response>.PrimaryShardReference primaryShardReference,
|
|
|
- boolean executeOnReplicas) {
|
|
|
+ protected ReplicationOperation<Request, Request, TransportReplicationAction.PrimaryResult<Request, TestResponse>>
|
|
|
+ createReplicatedOperation(
|
|
|
+ Request request,
|
|
|
+ ActionListener<TransportReplicationAction.PrimaryResult<Request, TestResponse>> actionListener,
|
|
|
+ TransportReplicationAction<Request, Request, TestResponse>.PrimaryShardReference primaryShardReference,
|
|
|
+ boolean executeOnReplicas) {
|
|
|
assertIndexShardCounter(1);
|
|
|
if (throwExceptionOnCreation) {
|
|
|
throw new ElasticsearchException("simulated exception, during createReplicatedOperation");
|
|
@@ -808,7 +785,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
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, shardStateAction,
|
|
|
+ TestAction action = new TestAction(Settings.EMPTY, "testActionWithExceptions", transportService, clusterService, shardStateAction,
|
|
|
threadPool) {
|
|
|
@Override
|
|
|
protected ReplicaResult shardOperationOnReplica(Request request, IndexShard replica) {
|
|
@@ -820,7 +797,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
return new ReplicaResult();
|
|
|
}
|
|
|
};
|
|
|
- final Action.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler();
|
|
|
+ final TestAction.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler();
|
|
|
try {
|
|
|
replicaOperationTransportHandler.messageReceived(
|
|
|
new TransportReplicationAction.ConcreteShardRequest<>(
|
|
@@ -871,7 +848,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
final String index = "test";
|
|
|
final ShardId shardId = new ShardId(index, "_na_", 0);
|
|
|
setState(clusterService, state(index, true, ShardRoutingState.STARTED));
|
|
|
- PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
|
|
+ PlainActionFuture<TestResponse> listener = new PlainActionFuture<>();
|
|
|
Request request = new Request(shardId).timeout("1ms");
|
|
|
action.new PrimaryOperationTransportHandler().messageReceived(
|
|
|
new TransportReplicationAction.ConcreteShardRequest<>(request, "_not_a_valid_aid_"),
|
|
@@ -897,7 +874,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
state = ClusterState.builder(state).nodes(DiscoveryNodes.builder(state.nodes()).localNodeId(replica.currentNodeId())).build();
|
|
|
setState(clusterService, state);
|
|
|
|
|
|
- PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
|
|
+ PlainActionFuture<TestResponse> listener = new PlainActionFuture<>();
|
|
|
Request request = new Request(shardId).timeout("1ms");
|
|
|
action.new ReplicaOperationTransportHandler().messageReceived(
|
|
|
new TransportReplicationAction.ConcreteShardRequest<>(request, "_not_a_valid_aid_"),
|
|
@@ -928,7 +905,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
setState(clusterService, state);
|
|
|
AtomicBoolean throwException = new AtomicBoolean(true);
|
|
|
final ReplicationTask task = maybeTask();
|
|
|
- Action action = new Action(Settings.EMPTY, "testActionWithExceptions", transportService, clusterService, shardStateAction,
|
|
|
+ TestAction action = new TestAction(Settings.EMPTY, "testActionWithExceptions", transportService, clusterService, shardStateAction,
|
|
|
threadPool) {
|
|
|
@Override
|
|
|
protected ReplicaResult shardOperationOnReplica(Request request, IndexShard replica) {
|
|
@@ -939,8 +916,8 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
return new ReplicaResult();
|
|
|
}
|
|
|
};
|
|
|
- final Action.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler();
|
|
|
- final PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
|
|
+ final TestAction.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler();
|
|
|
+ final PlainActionFuture<TestResponse> listener = new PlainActionFuture<>();
|
|
|
final Request request = new Request().setShardId(shardId);
|
|
|
request.primaryTerm(state.metaData().getIndexSafe(shardId.getIndex()).primaryTerm(shardId.id()));
|
|
|
replicaOperationTransportHandler.messageReceived(
|
|
@@ -1047,31 +1024,46 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- static class Response extends ReplicationResponse {
|
|
|
+ static class TestResponse extends ReplicationResponse {
|
|
|
}
|
|
|
|
|
|
- class Action extends TransportReplicationAction<Request, Request, Response> {
|
|
|
+ private class TestAction extends TransportReplicationAction<Request, Request, TestResponse> {
|
|
|
|
|
|
- Action(Settings settings, String actionName, TransportService transportService,
|
|
|
- ClusterService clusterService,
|
|
|
- ShardStateAction shardStateAction,
|
|
|
- ThreadPool threadPool) {
|
|
|
+ private final boolean withDocumentFailureOnPrimary;
|
|
|
+ private final boolean withDocumentFailureOnReplica;
|
|
|
+
|
|
|
+ TestAction(Settings settings, String actionName, TransportService transportService,
|
|
|
+ ClusterService clusterService, ShardStateAction shardStateAction,
|
|
|
+ ThreadPool threadPool) {
|
|
|
+ super(settings, actionName, transportService, clusterService, mockIndicesService(clusterService), threadPool,
|
|
|
+ shardStateAction,
|
|
|
+ new ActionFilters(new HashSet<>()), new IndexNameExpressionResolver(Settings.EMPTY),
|
|
|
+ Request::new, Request::new, ThreadPool.Names.SAME);
|
|
|
+ this.withDocumentFailureOnPrimary = false;
|
|
|
+ this.withDocumentFailureOnReplica = false;
|
|
|
+ }
|
|
|
+
|
|
|
+ TestAction(Settings settings, String actionName, TransportService transportService,
|
|
|
+ ClusterService clusterService, ShardStateAction shardStateAction,
|
|
|
+ ThreadPool threadPool, boolean withDocumentFailureOnPrimary, boolean withDocumentFailureOnReplica) {
|
|
|
super(settings, actionName, transportService, clusterService, mockIndicesService(clusterService), threadPool,
|
|
|
shardStateAction,
|
|
|
new ActionFilters(new HashSet<>()), new IndexNameExpressionResolver(Settings.EMPTY),
|
|
|
Request::new, Request::new, ThreadPool.Names.SAME);
|
|
|
+ this.withDocumentFailureOnPrimary = withDocumentFailureOnPrimary;
|
|
|
+ this.withDocumentFailureOnReplica = withDocumentFailureOnReplica;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- protected Response newResponseInstance() {
|
|
|
- return new Response();
|
|
|
+ protected TestResponse newResponseInstance() {
|
|
|
+ return new TestResponse();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
protected PrimaryResult shardOperationOnPrimary(Request shardRequest, IndexShard primary) throws Exception {
|
|
|
boolean executedBefore = shardRequest.processedOnPrimary.getAndSet(true);
|
|
|
assert executedBefore == false : "request has already been executed on the primary";
|
|
|
- return new PrimaryResult(shardRequest, new Response());
|
|
|
+ return new PrimaryResult(shardRequest, new TestResponse());
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -1156,22 +1148,23 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
return indexShard;
|
|
|
}
|
|
|
|
|
|
- class NoopReplicationOperation extends ReplicationOperation<Request, Request, Action.PrimaryResult<Request, Response>> {
|
|
|
- NoopReplicationOperation(Request request, ActionListener<Action.PrimaryResult<Request, Response>> listener) {
|
|
|
+ class NoopReplicationOperation extends ReplicationOperation<Request, Request, TestAction.PrimaryResult<Request, TestResponse>> {
|
|
|
+
|
|
|
+ NoopReplicationOperation(Request request, ActionListener<TestAction.PrimaryResult<Request, TestResponse>> listener) {
|
|
|
super(request, null, listener, true, null, null, TransportReplicationActionTests.this.logger, "noop");
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public void execute() throws Exception {
|
|
|
// Using the diamond operator (<>) prevents Eclipse from being able to compile this code
|
|
|
- this.resultListener.onResponse(new TransportReplicationAction.PrimaryResult<Request, Response>(null, new Response()));
|
|
|
+ this.resultListener.onResponse(new TransportReplicationAction.PrimaryResult<Request, TestResponse>(null, new TestResponse()));
|
|
|
}
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Transport channel that is needed for replica operation testing.
|
|
|
*/
|
|
|
- public TransportChannel createTransportChannel(final PlainActionFuture<Response> listener) {
|
|
|
+ public TransportChannel createTransportChannel(final PlainActionFuture<TestResponse> listener) {
|
|
|
return new TransportChannel() {
|
|
|
|
|
|
@Override
|
|
@@ -1186,12 +1179,12 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|
|
|
|
|
@Override
|
|
|
public void sendResponse(TransportResponse response) throws IOException {
|
|
|
- listener.onResponse(((Response) response));
|
|
|
+ listener.onResponse(((TestResponse) response));
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public void sendResponse(TransportResponse response, TransportResponseOptions options) throws IOException {
|
|
|
- listener.onResponse(((Response) response));
|
|
|
+ listener.onResponse(((TestResponse) response));
|
|
|
}
|
|
|
|
|
|
@Override
|