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
This commit is contained in:
parent
833fc8420f
commit
48b4f086e0
|
@ -549,8 +549,9 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
|
|||
public void handleException(TransportException exp) {
|
||||
try {
|
||||
// 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);
|
||||
retry(exp);
|
||||
} else {
|
||||
|
@ -799,6 +800,12 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
|
|||
protected IndexShardReference getIndexShardReferenceOnPrimary(ShardId shardId) {
|
||||
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
|
||||
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);
|
||||
}
|
||||
|
||||
|
|
|
@ -974,7 +974,6 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
|
||||
private void verifyPrimary() {
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -50,6 +50,8 @@ import org.elasticsearch.common.io.stream.StreamInput;
|
|||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
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.IndexShardState;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
@ -158,7 +160,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
ReplicationTask task = maybeTask();
|
||||
|
||||
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));
|
||||
TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
|
||||
reroutePhase.run();
|
||||
|
@ -166,7 +168,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
assertPhase(task, "failed");
|
||||
|
||||
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));
|
||||
listener = new PlainActionFuture<>();
|
||||
reroutePhase = action.new ReroutePhase(task, new Request().timeout("5ms"), listener);
|
||||
|
@ -181,7 +183,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
assertPhase(task, "waiting_for_retry");
|
||||
|
||||
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));
|
||||
assertListenerThrows("primary phase should fail operation when moving from a retryable block to a non-retryable one", listener, ClusterBlockException.class);
|
||||
assertIndexShardUninitialized();
|
||||
|
@ -196,7 +198,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
final ShardId shardId = new ShardId(index, "_na_", 0);
|
||||
// no replicas in oder to skip the replication part
|
||||
setState(clusterService, state(index, true,
|
||||
randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED));
|
||||
randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED));
|
||||
ReplicationTask task = maybeTask();
|
||||
|
||||
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 String primaryNodeId = shardRoutingTable.primaryShard().currentNodeId();
|
||||
final List<CapturingTransport.CapturedRequest> capturedRequests =
|
||||
transport.getCapturedRequestsByTargetNodeAndClear().get(primaryNodeId);
|
||||
transport.getCapturedRequestsByTargetNodeAndClear().get(primaryNodeId);
|
||||
assertThat(capturedRequests, notNullValue());
|
||||
assertThat(capturedRequests.size(), equalTo(1));
|
||||
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
|
||||
* relocation target and relocation target believes active primary to be on relocation source. This results in replication
|
||||
* 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
|
||||
* 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());
|
||||
final String primaryNodeId = shardRoutingTable.primaryShard().currentNodeId();
|
||||
final List<CapturingTransport.CapturedRequest> capturedRequests =
|
||||
transport.getCapturedRequestsByTargetNodeAndClear().get(primaryNodeId);
|
||||
transport.getCapturedRequestsByTargetNodeAndClear().get(primaryNodeId);
|
||||
assertThat(capturedRequests, notNullValue());
|
||||
assertThat(capturedRequests.size(), equalTo(1));
|
||||
assertThat(capturedRequests.get(0).action, equalTo("testAction[p]"));
|
||||
|
@ -282,7 +284,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
final String index = "test";
|
||||
// no replicas in oder to skip the replication part
|
||||
setState(clusterService, state(index, true,
|
||||
randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED));
|
||||
randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED));
|
||||
logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint());
|
||||
Request request = new Request(new ShardId("unknown_index", "_na_", 0)).timeout("1ms");
|
||||
PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
||||
|
@ -299,6 +301,61 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
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() {
|
||||
final String index = "test";
|
||||
final ShardId shardId = new ShardId(index, "_na_", 0);
|
||||
|
@ -449,7 +506,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
||||
ReplicationTask task = maybeTask();
|
||||
TransportReplicationAction<Request, Request, Response>.PrimaryPhase primaryPhase = actionWithRelocatingReplicasAfterPrimaryOp.new PrimaryPhase(
|
||||
task, request, createTransportChannel(listener));
|
||||
task, request, createTransportChannel(listener));
|
||||
primaryPhase.run();
|
||||
assertThat("request was not processed on primary", request.processedOnPrimary.get(), equalTo(true));
|
||||
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<>();
|
||||
ReplicationTask task = maybeTask();
|
||||
TransportReplicationAction<Request, Request, Response>.PrimaryPhase primaryPhase = actionWithDeletedIndexAfterPrimaryOp.new PrimaryPhase(
|
||||
task, request, createTransportChannel(listener));
|
||||
task, request, createTransportChannel(listener));
|
||||
primaryPhase.run();
|
||||
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));
|
||||
|
@ -529,8 +586,8 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
|
||||
setState(clusterService, state(index, true, ShardRoutingState.STARTED, replicaStates));
|
||||
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());
|
||||
PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
||||
|
@ -646,7 +703,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
|
||||
TransportChannel channel = createTransportChannel(listener, error::set);
|
||||
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.pending(), equalTo(assignedReplicas));
|
||||
|
@ -656,7 +713,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
|
||||
HashMap<String, Request> nodesSentTo = new HashMap<>();
|
||||
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) {
|
||||
// no duplicate requests
|
||||
Request replicationRequest = (Request) capturedRequest.request;
|
||||
|
@ -819,7 +876,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
final ShardId shardId = new ShardId(index, "_na_", 0);
|
||||
// one replica to make sure replication is attempted
|
||||
setState(clusterService, state(index, true,
|
||||
ShardRoutingState.STARTED, ShardRoutingState.STARTED));
|
||||
ShardRoutingState.STARTED, ShardRoutingState.STARTED));
|
||||
ShardRouting primaryShard = clusterService.state().routingTable().shardRoutingTable(shardId).primaryShard();
|
||||
indexShardRouting.set(primaryShard);
|
||||
logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint());
|
||||
|
@ -856,7 +913,7 @@ 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));
|
||||
ShardRoutingState.STARTED, ShardRoutingState.STARTED));
|
||||
action = new ActionWithDelay(Settings.EMPTY, "testActionWithExceptions", transportService, clusterService, threadPool);
|
||||
final Action.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler();
|
||||
final ReplicationTask task = maybeTask();
|
||||
|
@ -895,7 +952,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, ShardRoutingState.STARTED));
|
||||
ShardRoutingState.STARTED, ShardRoutingState.STARTED));
|
||||
logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint());
|
||||
Request request = new Request(shardId).timeout("100ms");
|
||||
PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
||||
|
@ -915,7 +972,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
final ShardId shardId = new ShardId(index, "_na_", 0);
|
||||
boolean localPrimary = true;
|
||||
setState(clusterService, state(index, localPrimary,
|
||||
ShardRoutingState.STARTED, ShardRoutingState.STARTED));
|
||||
ShardRoutingState.STARTED, ShardRoutingState.STARTED));
|
||||
Action action = new Action(Settings.EMPTY, "testAction", transportService, clusterService, threadPool) {
|
||||
@Override
|
||||
protected void resolveRequest(MetaData metaData, String concreteIndex, Request request) {
|
||||
|
@ -967,7 +1024,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
// publish a new cluster state
|
||||
boolean localPrimaryOnRetry = randomBoolean();
|
||||
setState(clusterService, state(index, localPrimaryOnRetry,
|
||||
ShardRoutingState.STARTED, ShardRoutingState.STARTED));
|
||||
ShardRoutingState.STARTED, ShardRoutingState.STARTED));
|
||||
CapturingTransport.CapturedRequest[] primaryRetry = transport.getCapturedRequestsAndClear();
|
||||
|
||||
// the request should be retried
|
||||
|
@ -1083,8 +1140,8 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
ClusterService clusterService,
|
||||
ThreadPool 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
|
||||
|
|
Loading…
Reference in New Issue