From aecc31a8571d3e628d93d9ae142867e0af6379c1 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 27 Jul 2018 17:22:13 +0200 Subject: [PATCH 01/17] fix promotion race --- .../elasticsearch/index/shard/IndexShard.java | 81 ++++++++++--------- .../shard/IndexShardOperationPermits.java | 34 ++++---- .../IndexLevelReplicationTests.java | 48 +++++++++++ .../IndexShardOperationPermitsTests.java | 40 ++++++++- .../ESIndexLevelReplicationTestCase.java | 43 ++++++---- 5 files changed, 172 insertions(+), 74 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index d4a1d0502d0aa..197cd35b345e9 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -473,6 +473,8 @@ public void updateShardState(final ShardRouting newRouting, TimeUnit.MINUTES, () -> { shardStateUpdated.await(); + assert primaryTerm == newPrimaryTerm : + "shard term changed on primary. expected [" + newPrimaryTerm + "] but was [" + primaryTerm + "]"; try { /* * If this shard was serving as a replica shard when another shard was promoted to primary then the state of @@ -2216,51 +2218,56 @@ public void acquireReplicaOperationPermit(final long operationPrimaryTerm, final final Object debugInfo) { verifyNotClosed(); verifyReplicationTarget(); - final boolean globalCheckpointUpdated; if (operationPrimaryTerm > primaryTerm) { synchronized (primaryTermMutex) { if (operationPrimaryTerm > primaryTerm) { + verifyNotClosed(); + IndexShardState shardState = state(); // only roll translog and update primary term if shard has made it past recovery // Having a new primary term here means that the old primary failed and that there is a new primary, which again // means that the master will fail this shard as all initializing shards are failed when a primary is selected - // We abort early here to prevent an ongoing recovery from the failed primary to mess with the global / local checkpoint + // We abort early here to prevent an ongoing recovery from the failed primary to mess with the global / local + // checkpoint if (shardState != IndexShardState.POST_RECOVERY && shardState != IndexShardState.STARTED) { throw new IndexShardNotStartedException(shardId, shardState); } - try { - indexShardOperationPermits.blockOperations(30, TimeUnit.MINUTES, () -> { - assert operationPrimaryTerm > primaryTerm : - "shard term already update. op term [" + operationPrimaryTerm + "], shardTerm [" + primaryTerm + "]"; + + synchronized (mutex) { + final CountDownLatch termUpdated = new CountDownLatch(1); + if (operationPrimaryTerm > primaryTerm) { + indexShardOperationPermits.asyncBlockOperations(30, TimeUnit.MINUTES, () -> { + termUpdated.await(); + // a primary promotion, or another primary term transition, might have been triggered concurrently to this + // recheck under the operation permit if we can skip doing this work + if (operationPrimaryTerm == primaryTerm) { + updateGlobalCheckpointOnReplica(globalCheckpoint, "primary term transition"); + final long currentGlobalCheckpoint = getGlobalCheckpoint(); + final long localCheckpoint; + if (currentGlobalCheckpoint == SequenceNumbers.UNASSIGNED_SEQ_NO) { + localCheckpoint = SequenceNumbers.NO_OPS_PERFORMED; + } else { + localCheckpoint = currentGlobalCheckpoint; + } + logger.trace( + "detected new primary with primary term [{}], resetting local checkpoint from [{}] to [{}]", + operationPrimaryTerm, + getLocalCheckpoint(), + localCheckpoint); + getEngine().resetLocalCheckpoint(localCheckpoint); + getEngine().rollTranslogGeneration(); + } else { + logger.trace("a primary promotion or concurrent primary term transition has made this reset obsolete"); + } + }, e -> failShard("exception during primary term transition", e)); + primaryTerm = operationPrimaryTerm; - updateGlobalCheckpointOnReplica(globalCheckpoint, "primary term transition"); - final long currentGlobalCheckpoint = getGlobalCheckpoint(); - final long localCheckpoint; - if (currentGlobalCheckpoint == SequenceNumbers.UNASSIGNED_SEQ_NO) { - localCheckpoint = SequenceNumbers.NO_OPS_PERFORMED; - } else { - localCheckpoint = currentGlobalCheckpoint; - } - logger.trace( - "detected new primary with primary term [{}], resetting local checkpoint from [{}] to [{}]", - operationPrimaryTerm, - getLocalCheckpoint(), - localCheckpoint); - getEngine().resetLocalCheckpoint(localCheckpoint); - getEngine().rollTranslogGeneration(); - }); - globalCheckpointUpdated = true; - } catch (final Exception e) { - onPermitAcquired.onFailure(e); - return; + termUpdated.countDown(); + } } - } else { - globalCheckpointUpdated = false; } } - } else { - globalCheckpointUpdated = false; } assert operationPrimaryTerm <= primaryTerm @@ -2279,14 +2286,12 @@ public void onResponse(final Releasable releasable) { primaryTerm); onPermitAcquired.onFailure(new IllegalStateException(message)); } else { - if (globalCheckpointUpdated == false) { - try { - updateGlobalCheckpointOnReplica(globalCheckpoint, "operation"); - } catch (Exception e) { - releasable.close(); - onPermitAcquired.onFailure(e); - return; - } + try { + updateGlobalCheckpointOnReplica(globalCheckpoint, "operation"); + } catch (Exception e) { + releasable.close(); + onPermitAcquired.onFailure(e); + return; } onPermitAcquired.onResponse(releasable); } diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShardOperationPermits.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShardOperationPermits.java index a0d46c8eb2330..bddb8be681c1f 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShardOperationPermits.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShardOperationPermits.java @@ -33,6 +33,7 @@ import java.io.Closeable; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -59,7 +60,7 @@ final class IndexShardOperationPermits implements Closeable { final Semaphore semaphore = new Semaphore(TOTAL_PERMITS, true); // fair to ensure a blocking thread is not starved private final List delayedOperations = new ArrayList<>(); // operations that are delayed private volatile boolean closed; - private boolean delayed; // does not need to be volatile as all accesses are done under a lock on this + private int queuedBlockOperations; // does not need to be volatile as all accesses are done under a lock on this // only valid when assertions are enabled. Key is AtomicBoolean associated with each permit to ensure close once semantics. // Value is a tuple, with a some debug information supplied by the caller and a stack trace of the acquiring thread @@ -102,9 +103,6 @@ void blockOperations( final long timeout, final TimeUnit timeUnit, final CheckedRunnable onBlocked) throws InterruptedException, TimeoutException, E { - if (closed) { - throw new IndexShardClosedException(shardId); - } delayOperations(); try { doBlockOperations(timeout, timeUnit, onBlocked); @@ -128,6 +126,7 @@ void blockOperations( void asyncBlockOperations( final long timeout, final TimeUnit timeUnit, final CheckedRunnable onBlocked, final Consumer onFailure) { delayOperations(); + threadPool.executor(ThreadPool.Names.GENERIC).execute(new AbstractRunnable() { @Override public void onFailure(final Exception e) { @@ -147,13 +146,12 @@ public void onAfter() { } private void delayOperations() { + if (closed) { + throw new IndexShardClosedException(shardId); + } synchronized (this) { - if (delayed) { - throw new IllegalStateException("operations are already delayed"); - } else { - assert delayedOperations.isEmpty(); - delayed = true; - } + assert queuedBlockOperations > 0 || delayedOperations.isEmpty(); + queuedBlockOperations++; } } @@ -164,7 +162,7 @@ private void doBlockOperations( if (Assertions.ENABLED) { // since delayed is not volatile, we have to synchronize even here for visibility synchronized (this) { - assert delayed; + assert queuedBlockOperations > 0; } } if (semaphore.tryAcquire(TOTAL_PERMITS, timeout, timeUnit)) { @@ -182,10 +180,14 @@ private void doBlockOperations( private void releaseDelayedOperations() { final List queuedActions; synchronized (this) { - assert delayed; - queuedActions = new ArrayList<>(delayedOperations); - delayedOperations.clear(); - delayed = false; + assert queuedBlockOperations > 0; + queuedBlockOperations--; + if (queuedBlockOperations == 0) { + queuedActions = new ArrayList<>(delayedOperations); + delayedOperations.clear(); + } else { + queuedActions = Collections.emptyList(); + } } if (!queuedActions.isEmpty()) { /* @@ -242,7 +244,7 @@ private void acquire(final ActionListener onAcquired, final String e final Releasable releasable; try { synchronized (this) { - if (delayed) { + if (queuedBlockOperations > 0) { final Supplier contextSupplier = threadPool.getThreadContext().newRestorableContext(false); final ActionListener wrappedListener; if (executorOnDelay != null) { diff --git a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java index b05b1e5cc5ca0..f299061671b7b 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java @@ -59,6 +59,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CyclicBarrier; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -234,6 +235,53 @@ public void testConflictingOpsOnReplica() throws Exception { } } + public void testReplicaTermIncrementWithConcurrentPrimaryPromotion() throws Exception { + Map mappings = + Collections.singletonMap("type", "{ \"type\": { \"properties\": { \"f\": { \"type\": \"keyword\"} }}}"); + try (ReplicationGroup shards = new ReplicationGroup(buildIndexMetaData(2, mappings))) { + shards.startAll(); + long primaryPrimaryTerm = shards.getPrimary().getPrimaryTerm(); + List replicas = shards.getReplicas(); + IndexShard replica1 = replicas.get(0); + IndexShard replica2 = replicas.get(1); + + shards.promoteReplicaToPrimary(replica1, (shard, listener) -> {}); + long newReplica1Term = replica1.getPrimaryTerm(); + assertEquals(primaryPrimaryTerm + 1, newReplica1Term); + + assertEquals(primaryPrimaryTerm, replica2.getPrimaryTerm()); + + IndexRequest indexRequest = new IndexRequest(index.getName(), "type", "1").source("{ \"f\": \"1\"}", XContentType.JSON); + BulkShardRequest replicationRequest = indexOnPrimary(indexRequest, replica1); + + CyclicBarrier barrier = new CyclicBarrier(2); + Thread t1 = new Thread(() -> { + try { + barrier.await(); + indexOnReplica(replicationRequest, shards, replica2, newReplica1Term); + } catch (IllegalStateException ise) { + assertThat(ise.getMessage(), containsString("is too old")); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + Thread t2 = new Thread(() -> { + try { + barrier.await(); + shards.promoteReplicaToPrimary(replica2).get(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + t2.start(); + t1.start(); + t1.join(); + t2.join(); + + assertEquals(newReplica1Term + 1, replica2.getPrimaryTerm()); + } + } + /** * test document failures (failures after seq_no generation) are added as noop operation to the translog * for primary and replica shards diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardOperationPermitsTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardOperationPermitsTests.java index 27d08b76c0310..23337def2ae1b 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardOperationPermitsTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardOperationPermitsTests.java @@ -71,7 +71,7 @@ public class IndexShardOperationPermitsTests extends ESTestCase { public static void setupThreadPool() { int writeThreadPoolSize = randomIntBetween(1, 2); int writeThreadPoolQueueSize = randomIntBetween(1, 2); - threadPool = new TestThreadPool("IndexShardOperationsLockTests", + threadPool = new TestThreadPool("IndexShardOperationPermitsTests", Settings.builder() .put("thread_pool." + ThreadPool.Names.WRITE + ".size", writeThreadPoolSize) .put("thread_pool." + ThreadPool.Names.WRITE + ".queue_size", writeThreadPoolQueueSize) @@ -100,7 +100,7 @@ public void checkNoInflightOperations() { assertThat(permits.getActiveOperationsCount(), equalTo(0)); } - public void testAllOperationsInvoked() throws InterruptedException, TimeoutException, ExecutionException { + public void testAllOperationsInvoked() throws InterruptedException, TimeoutException { int numThreads = 10; class DummyException extends RuntimeException {} @@ -187,7 +187,7 @@ public void testOperationsInvokedImmediatelyIfNoBlock() throws ExecutionExceptio future.get().close(); } - public void testOperationsIfClosed() throws ExecutionException, InterruptedException { + public void testOperationsIfClosed() { PlainActionFuture future = new PlainActionFuture<>(); permits.close(); permits.acquire(future, ThreadPool.Names.GENERIC, true, ""); @@ -195,10 +195,12 @@ public void testOperationsIfClosed() throws ExecutionException, InterruptedExcep assertThat(exception.getCause(), instanceOf(IndexShardClosedException.class)); } - public void testBlockIfClosed() throws ExecutionException, InterruptedException { + public void testBlockIfClosed() { permits.close(); expectThrows(IndexShardClosedException.class, () -> permits.blockOperations(randomInt(10), TimeUnit.MINUTES, () -> { throw new IllegalArgumentException("fake error"); })); + expectThrows(IndexShardClosedException.class, () -> permits.asyncBlockOperations(randomInt(10), TimeUnit.MINUTES, + () -> { throw new IllegalArgumentException("fake error"); }, e -> { throw new AssertionError(e); })); } public void testOperationsDelayedIfBlock() throws ExecutionException, InterruptedException, TimeoutException { @@ -210,6 +212,36 @@ public void testOperationsDelayedIfBlock() throws ExecutionException, Interrupte future.get(1, TimeUnit.HOURS).close(); } + public void testGetBlockWhenBlocked() throws ExecutionException, InterruptedException, TimeoutException { + PlainActionFuture future = new PlainActionFuture<>(); + final CountDownLatch blockAcquired = new CountDownLatch(1); + final CountDownLatch releaseBlock = new CountDownLatch(1); + final AtomicBoolean blocked = new AtomicBoolean(); + try (Releasable ignored = blockAndWait()) { + permits.acquire(future, ThreadPool.Names.GENERIC, true, ""); + + permits.asyncBlockOperations( + 30, + TimeUnit.MINUTES, + () -> { + blocked.set(true); + blockAcquired.countDown(); + releaseBlock.await(); + }, + e -> { + throw new RuntimeException(e); + }); + assertFalse(blocked.get()); + assertFalse(future.isDone()); + } + blockAcquired.await(); + assertTrue(blocked.get()); + assertFalse(future.isDone()); + releaseBlock.countDown(); + + future.get(1, TimeUnit.HOURS).close(); + } + /** * Tests that the ThreadContext is restored when a operation is executed after it has been delayed due to a block */ diff --git a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index 2b1841c39ae52..026b882c98c58 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -84,6 +84,7 @@ import java.util.concurrent.Future; import java.util.concurrent.FutureTask; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.function.Consumer; import java.util.function.Function; @@ -279,20 +280,10 @@ public synchronized List getReplicas() { /** * promotes the specific replica as the new primary */ - public synchronized Future promoteReplicaToPrimary(IndexShard replica) throws IOException { - final long newTerm = indexMetaData.primaryTerm(shardId.id()) + 1; - IndexMetaData.Builder newMetaData = IndexMetaData.builder(indexMetaData).primaryTerm(shardId.id(), newTerm); - indexMetaData = newMetaData.build(); - assertTrue(replicas.remove(replica)); - closeShards(primary); - primary = replica; - assert primary.routingEntry().active() : "only active replicas can be promoted to primary: " + primary.routingEntry(); + public Future promoteReplicaToPrimary(IndexShard replica) throws IOException { PlainActionFuture fut = new PlainActionFuture<>(); - ShardRouting primaryRouting = replica.routingEntry().moveActiveReplicaToPrimary(); - IndexShardRoutingTable routingTable = routingTable(shr -> shr == replica.routingEntry() ? primaryRouting : shr); - - primary.updateShardState(primaryRouting, - newTerm, (shard, listener) -> primaryReplicaSyncer.resync(shard, + promoteReplicaToPrimary(replica, + (shard, listener) -> primaryReplicaSyncer.resync(shard, new ActionListener() { @Override public void onResponse(PrimaryReplicaSyncer.ResyncTask resyncTask) { @@ -305,11 +296,27 @@ public void onFailure(Exception e) { listener.onFailure(e); fut.onFailure(e); } - }), currentClusterStateVersion.incrementAndGet(), activeIds(), routingTable, Collections.emptySet()); - + })); return fut; } + public synchronized void promoteReplicaToPrimary(IndexShard replica, + BiConsumer> primaryReplicaSyncer) + throws IOException { + final long newTerm = indexMetaData.primaryTerm(shardId.id()) + 1; + IndexMetaData.Builder newMetaData = IndexMetaData.builder(indexMetaData).primaryTerm(shardId.id(), newTerm); + indexMetaData = newMetaData.build(); + assertTrue(replicas.remove(replica)); + closeShards(primary); + primary = replica; + assert primary.routingEntry().active() : "only active replicas can be promoted to primary: " + primary.routingEntry(); + ShardRouting primaryRouting = replica.routingEntry().moveActiveReplicaToPrimary(); + IndexShardRoutingTable routingTable = routingTable(shr -> shr == replica.routingEntry() ? primaryRouting : shr); + + primary.updateShardState(primaryRouting, newTerm, primaryReplicaSyncer, currentClusterStateVersion.incrementAndGet(), + activeIds(), routingTable, Collections.emptySet()); + } + private synchronized Set activeIds() { return shardRoutings().stream() .filter(ShardRouting::active).map(ShardRouting::allocationId).map(AllocationId::getId).collect(Collectors.toSet()); @@ -664,7 +671,11 @@ BulkShardRequest deleteOnPrimary(DeleteRequest request, IndexShard primary) thro * indexes the given requests on the supplied replica shard */ void indexOnReplica(BulkShardRequest request, ReplicationGroup group, IndexShard replica) throws Exception { - executeShardBulkOnReplica(request, replica, group.primary.getPrimaryTerm(), group.primary.getGlobalCheckpoint()); + indexOnReplica(request, group, replica, group.primary.getPrimaryTerm()); + } + + void indexOnReplica(BulkShardRequest request, ReplicationGroup group, IndexShard replica, long term) throws Exception { + executeShardBulkOnReplica(request, replica, term, group.primary.getGlobalCheckpoint()); } /** From 9396f53af3d4fb1816dfd35ae3222d622ff3aa9e Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 27 Jul 2018 19:29:59 +0200 Subject: [PATCH 02/17] undo extra line breaks --- .../main/java/org/elasticsearch/index/shard/IndexShard.java | 3 +-- .../elasticsearch/index/shard/IndexShardOperationPermits.java | 1 - 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 197cd35b345e9..703e397fabcc3 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -2227,8 +2227,7 @@ public void acquireReplicaOperationPermit(final long operationPrimaryTerm, final // only roll translog and update primary term if shard has made it past recovery // Having a new primary term here means that the old primary failed and that there is a new primary, which again // means that the master will fail this shard as all initializing shards are failed when a primary is selected - // We abort early here to prevent an ongoing recovery from the failed primary to mess with the global / local - // checkpoint + // We abort early here to prevent an ongoing recovery from the failed primary to mess with the global / local checkpoint if (shardState != IndexShardState.POST_RECOVERY && shardState != IndexShardState.STARTED) { throw new IndexShardNotStartedException(shardId, shardState); diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShardOperationPermits.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShardOperationPermits.java index bddb8be681c1f..09c7a04762081 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShardOperationPermits.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShardOperationPermits.java @@ -126,7 +126,6 @@ void blockOperations( void asyncBlockOperations( final long timeout, final TimeUnit timeUnit, final CheckedRunnable onBlocked, final Consumer onFailure) { delayOperations(); - threadPool.executor(ThreadPool.Names.GENERIC).execute(new AbstractRunnable() { @Override public void onFailure(final Exception e) { From a102ef9fdcba0ff3cfec83650257ebffbdbc88e5 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Sun, 29 Jul 2018 14:38:03 +0200 Subject: [PATCH 03/17] Distinguish between operation and cluster state term --- .../action/bulk/TransportShardBulkAction.java | 9 ++- .../TransportReplicationAction.java | 2 +- .../elasticsearch/index/shard/IndexShard.java | 81 +++++++++++-------- .../index/shard/PrimaryReplicaSyncer.java | 2 +- .../index/shard/StoreRecovery.java | 11 +-- .../indices/recovery/RecoveryTarget.java | 2 +- .../TransportReplicationActionTests.java | 8 +- .../TransportWriteActionTests.java | 4 +- .../routing/allocation/ShardStateIT.java | 2 +- .../IndexLevelReplicationTests.java | 12 +-- .../RecoveryDuringReplicationTests.java | 3 +- .../index/shard/IndexShardTests.java | 52 ++++++------ .../shard/PrimaryReplicaSyncerTests.java | 4 +- .../indices/recovery/RecoveryTests.java | 2 +- .../ESIndexLevelReplicationTestCase.java | 12 +-- .../index/shard/IndexShardTestCase.java | 10 +-- 16 files changed, 116 insertions(+), 100 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 15a98077eac4a..7e16d2d7c43b7 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -144,7 +144,7 @@ private static BulkItemResultHolder executeIndexRequest(final IndexRequest index switch (indexResult.getResultType()) { case SUCCESS: IndexResponse response = new IndexResponse(primary.shardId(), indexRequest.type(), indexRequest.id(), - indexResult.getSeqNo(), primary.getPrimaryTerm(), indexResult.getVersion(), indexResult.isCreated()); + indexResult.getSeqNo(), primary.getOperationPrimaryTerm(), indexResult.getVersion(), indexResult.isCreated()); return new BulkItemResultHolder(response, indexResult, bulkItemRequest); case FAILURE: return new BulkItemResultHolder(null, indexResult, bulkItemRequest); @@ -161,7 +161,7 @@ private static BulkItemResultHolder executeDeleteRequest(final DeleteRequest del switch (deleteResult.getResultType()) { case SUCCESS: DeleteResponse response = new DeleteResponse(primary.shardId(), deleteRequest.type(), deleteRequest.id(), - deleteResult.getSeqNo(), primary.getPrimaryTerm(), deleteResult.getVersion(), deleteResult.isFound()); + deleteResult.getSeqNo(), primary.getOperationPrimaryTerm(), deleteResult.getVersion(), deleteResult.isFound()); return new BulkItemResultHolder(response, deleteResult, bulkItemRequest); case FAILURE: return new BulkItemResultHolder(null, deleteResult, bulkItemRequest); @@ -300,7 +300,7 @@ static BulkItemResultHolder processUpdateResponse(final UpdateRequest updateRequ assert result instanceof Engine.IndexResult : result.getClass(); final IndexRequest updateIndexRequest = translate.action(); final IndexResponse indexResponse = new IndexResponse(primary.shardId(), updateIndexRequest.type(), updateIndexRequest.id(), - result.getSeqNo(), primary.getPrimaryTerm(), result.getVersion(), ((Engine.IndexResult) result).isCreated()); + result.getSeqNo(), primary.getOperationPrimaryTerm(), result.getVersion(), ((Engine.IndexResult) result).isCreated()); updateResponse = new UpdateResponse(indexResponse.getShardInfo(), indexResponse.getShardId(), indexResponse.getType(), indexResponse.getId(), indexResponse.getSeqNo(), indexResponse.getPrimaryTerm(), indexResponse.getVersion(), indexResponse.getResult()); @@ -320,7 +320,7 @@ static BulkItemResultHolder processUpdateResponse(final UpdateRequest updateRequ final DeleteRequest updateDeleteRequest = translate.action(); final DeleteResponse deleteResponse = new DeleteResponse(primary.shardId(), updateDeleteRequest.type(), updateDeleteRequest.id(), - result.getSeqNo(), primary.getPrimaryTerm(), result.getVersion(), ((Engine.DeleteResult) result).isFound()); + result.getSeqNo(), primary.getOperationPrimaryTerm(), result.getVersion(), ((Engine.DeleteResult) result).isFound()); updateResponse = new UpdateResponse(deleteResponse.getShardInfo(), deleteResponse.getShardId(), deleteResponse.getType(), deleteResponse.getId(), deleteResponse.getSeqNo(), deleteResponse.getPrimaryTerm(), @@ -490,6 +490,7 @@ public static Translog.Location performOnReplica(BulkShardRequest request, Index switch (replicaItemExecutionMode(item, i)) { case NORMAL: final DocWriteResponse primaryResponse = item.getPrimaryResponse().getResponse(); + assert replica.getOperationPrimaryTerm() == primaryResponse.getPrimaryTerm(); operationResult = performOpOnReplica(primaryResponse, docWriteRequest, replica); assert operationResult != null : "operation result must never be null when primary response has no failure"; location = syncOperationResultOrThrow(operationResult, location); diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 53d9752f4edc6..5c3cfdfe08bb6 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -929,7 +929,7 @@ private void acquirePrimaryShardReference(ShardId shardId, String allocationId, if (actualAllocationId.equals(allocationId) == false) { throw new ShardNotFoundException(shardId, "expected aID [{}] but found [{}]", allocationId, actualAllocationId); } - final long actualTerm = indexShard.getPrimaryTerm(); + final long actualTerm = indexShard.getClusterStatePrimaryTerm(); if (actualTerm != primaryTerm) { throw new ShardNotFoundException(shardId, "expected aID [{}] with term [{}] but found [{}]", allocationId, primaryTerm, actualTerm); diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 703e397fabcc3..8d0300a3f0ffe 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -192,7 +192,8 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl protected volatile ShardRouting shardRouting; protected volatile IndexShardState state; - protected volatile long primaryTerm; + protected volatile long clusterStatePrimaryTerm; + protected volatile long operationPrimaryTerm; protected final AtomicReference currentEngineReference = new AtomicReference<>(); final EngineFactory engineFactory; @@ -315,7 +316,8 @@ public boolean shouldCache(Query query) { } indexShardOperationPermits = new IndexShardOperationPermits(shardId, threadPool); searcherWrapper = indexSearcherWrapper; - primaryTerm = indexSettings.getIndexMetaData().primaryTerm(shardId.id()); + clusterStatePrimaryTerm = indexSettings.getIndexMetaData().primaryTerm(shardId.id()); + operationPrimaryTerm = clusterStatePrimaryTerm; refreshListeners = buildRefreshListeners(); lastSearcherAccess.set(threadPool.relativeTimeInMillis()); persistMetadata(path, indexSettings, shardRouting, null, logger); @@ -367,8 +369,12 @@ public ShardFieldData fieldData() { /** * Returns the primary term the index shard is on. See {@link org.elasticsearch.cluster.metadata.IndexMetaData#primaryTerm(int)} */ - public long getPrimaryTerm() { - return this.primaryTerm; + public long getClusterStatePrimaryTerm() { + return this.clusterStatePrimaryTerm; + } + + public long getOperationPrimaryTerm() { + return this.operationPrimaryTerm; } /** @@ -431,7 +437,7 @@ public void updateShardState(final ShardRouting newRouting, final CountDownLatch shardStateUpdated = new CountDownLatch(1); if (newRouting.primary()) { - if (newPrimaryTerm == primaryTerm) { + if (newPrimaryTerm == clusterStatePrimaryTerm) { if (currentRouting.initializing() && currentRouting.isRelocationTarget() == false && newRouting.active()) { // the master started a recovering primary, activate primary mode. replicationTracker.activatePrimaryMode(getLocalCheckpoint()); @@ -454,10 +460,10 @@ public void updateShardState(final ShardRouting newRouting, assert newRouting.initializing() == false : "a started primary shard should never update its term; " + "shard " + newRouting + ", " - + "current term [" + primaryTerm + "], " + + "current term [" + clusterStatePrimaryTerm + "], " + "new term [" + newPrimaryTerm + "]"; - assert newPrimaryTerm > primaryTerm : - "primary terms can only go up; current term [" + primaryTerm + "], new term [" + newPrimaryTerm + "]"; + assert newPrimaryTerm > clusterStatePrimaryTerm : + "primary terms can only go up; current term [" + clusterStatePrimaryTerm + "], new term [" + newPrimaryTerm + "]"; /* * Before this call returns, we are guaranteed that all future operations are delayed and so this happens before we * increment the primary term. The latch is needed to ensure that we do not unblock operations before the primary term is @@ -473,8 +479,10 @@ public void updateShardState(final ShardRouting newRouting, TimeUnit.MINUTES, () -> { shardStateUpdated.await(); - assert primaryTerm == newPrimaryTerm : - "shard term changed on primary. expected [" + newPrimaryTerm + "] but was [" + primaryTerm + "]"; + assert clusterStatePrimaryTerm == newPrimaryTerm : + "shard term changed on primary. expected [" + newPrimaryTerm + "] but was [" + clusterStatePrimaryTerm + "]"; + assert operationPrimaryTerm < newPrimaryTerm; + operationPrimaryTerm = newPrimaryTerm; try { /* * If this shard was serving as a replica shard when another shard was promoted to primary then the state of @@ -522,7 +530,7 @@ public void onFailure(Exception e) { }, e -> failShard("exception during primary term transition", e)); replicationTracker.activatePrimaryMode(getLocalCheckpoint()); - primaryTerm = newPrimaryTerm; + clusterStatePrimaryTerm = newPrimaryTerm; } } // set this last, once we finished updating all internal state. @@ -657,21 +665,22 @@ private IndexShardState changeState(IndexShardState newState, String reason) { public Engine.IndexResult applyIndexOperationOnPrimary(long version, VersionType versionType, SourceToParse sourceToParse, long autoGeneratedTimestamp, boolean isRetry) throws IOException { assert versionType.validateVersionForWrites(version); - return applyIndexOperation(SequenceNumbers.UNASSIGNED_SEQ_NO, primaryTerm, version, versionType, autoGeneratedTimestamp, + return applyIndexOperation(SequenceNumbers.UNASSIGNED_SEQ_NO, operationPrimaryTerm, version, versionType, autoGeneratedTimestamp, isRetry, Engine.Operation.Origin.PRIMARY, sourceToParse); } public Engine.IndexResult applyIndexOperationOnReplica(long seqNo, long version, long autoGeneratedTimeStamp, boolean isRetry, SourceToParse sourceToParse) throws IOException { - return applyIndexOperation(seqNo, primaryTerm, version, null, autoGeneratedTimeStamp, isRetry, + return applyIndexOperation(seqNo, operationPrimaryTerm, version, null, autoGeneratedTimeStamp, isRetry, Engine.Operation.Origin.REPLICA, sourceToParse); } private Engine.IndexResult applyIndexOperation(long seqNo, long opPrimaryTerm, long version, @Nullable VersionType versionType, long autoGeneratedTimeStamp, boolean isRetry, Engine.Operation.Origin origin, SourceToParse sourceToParse) throws IOException { - assert opPrimaryTerm <= this.primaryTerm : "op term [ " + opPrimaryTerm + " ] > shard term [" + this.primaryTerm + "]"; + assert opPrimaryTerm <= this.operationPrimaryTerm: "op term [ " + opPrimaryTerm + " ] > shard term [" + this.operationPrimaryTerm + + "]"; ensureWriteAllowed(origin); Engine.Index operation; try { @@ -725,12 +734,13 @@ private Engine.IndexResult index(Engine engine, Engine.Index index) throws IOExc } public Engine.NoOpResult markSeqNoAsNoop(long seqNo, String reason) throws IOException { - return markSeqNoAsNoop(seqNo, primaryTerm, reason, Engine.Operation.Origin.REPLICA); + return markSeqNoAsNoop(seqNo, operationPrimaryTerm, reason, Engine.Operation.Origin.REPLICA); } private Engine.NoOpResult markSeqNoAsNoop(long seqNo, long opPrimaryTerm, String reason, Engine.Operation.Origin origin) throws IOException { - assert opPrimaryTerm <= this.primaryTerm : "op term [ " + opPrimaryTerm + " ] > shard term [" + this.primaryTerm + "]"; + assert opPrimaryTerm <= this.operationPrimaryTerm : "op term [ " + opPrimaryTerm + " ] > shard term [" + this.operationPrimaryTerm + + "]"; long startTime = System.nanoTime(); ensureWriteAllowed(origin); final Engine.NoOp noOp = new Engine.NoOp(seqNo, opPrimaryTerm, origin, startTime, reason); @@ -748,17 +758,18 @@ private Engine.NoOpResult noOp(Engine engine, Engine.NoOp noOp) { public Engine.DeleteResult applyDeleteOperationOnPrimary(long version, String type, String id, VersionType versionType) throws IOException { assert versionType.validateVersionForWrites(version); - return applyDeleteOperation(SequenceNumbers.UNASSIGNED_SEQ_NO, primaryTerm, version, type, id, versionType, + return applyDeleteOperation(SequenceNumbers.UNASSIGNED_SEQ_NO, operationPrimaryTerm, version, type, id, versionType, Engine.Operation.Origin.PRIMARY); } public Engine.DeleteResult applyDeleteOperationOnReplica(long seqNo, long version, String type, String id) throws IOException { - return applyDeleteOperation(seqNo, primaryTerm, version, type, id, null, Engine.Operation.Origin.REPLICA); + return applyDeleteOperation(seqNo, operationPrimaryTerm, version, type, id, null, Engine.Operation.Origin.REPLICA); } private Engine.DeleteResult applyDeleteOperation(long seqNo, long opPrimaryTerm, long version, String type, String id, @Nullable VersionType versionType, Engine.Operation.Origin origin) throws IOException { - assert opPrimaryTerm <= this.primaryTerm : "op term [ " + opPrimaryTerm + " ] > shard term [" + this.primaryTerm + "]"; + assert opPrimaryTerm <= this.operationPrimaryTerm : "op term [ " + opPrimaryTerm + " ] > shard term [" + this.operationPrimaryTerm + + "]"; ensureWriteAllowed(origin); // When there is a single type, the unique identifier is only composed of the _id, // so there is no way to differenciate foo#1 from bar#1. This is especially an issue @@ -1211,7 +1222,7 @@ public void prepareForIndexRecovery() { } public void trimOperationOfPreviousPrimaryTerms(long aboveSeqNo) { - getEngine().trimOperationsFromTranslog(primaryTerm, aboveSeqNo); + getEngine().trimOperationsFromTranslog(operationPrimaryTerm, aboveSeqNo); } public Engine.Result applyTranslogOperation(Translog.Operation operation, Engine.Operation.Origin origin) throws IOException { @@ -2177,7 +2188,7 @@ private EngineConfig newEngineConfig() { IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()), Collections.singletonList(refreshListeners), Collections.singletonList(new RefreshMetricUpdater(refreshMetric)), - indexSort, this::runTranslogRecovery, circuitBreakerService, replicationTracker, this::getPrimaryTerm); + indexSort, this::runTranslogRecovery, circuitBreakerService, replicationTracker, this::getOperationPrimaryTerm); } /** @@ -2205,7 +2216,7 @@ public void acquirePrimaryOperationPermit(ActionListener onPermitAcq * {@link IllegalStateException}. If permit acquisition is delayed, the listener will be invoked on the executor with the specified * name. * - * @param operationPrimaryTerm the operation primary term + * @param opPrimaryTerm the operation primary term * @param globalCheckpoint the global checkpoint associated with the request * @param onPermitAcquired the listener for permit acquisition * @param executorOnDelay the name of the executor to invoke the listener on if permit acquisition is delayed @@ -2213,14 +2224,14 @@ public void acquirePrimaryOperationPermit(ActionListener onPermitAcq * the tracing will capture the supplied object's {@link Object#toString()} value. Otherwise the object * isn't used */ - public void acquireReplicaOperationPermit(final long operationPrimaryTerm, final long globalCheckpoint, + public void acquireReplicaOperationPermit(final long opPrimaryTerm, final long globalCheckpoint, final ActionListener onPermitAcquired, final String executorOnDelay, final Object debugInfo) { verifyNotClosed(); verifyReplicationTarget(); - if (operationPrimaryTerm > primaryTerm) { + if (opPrimaryTerm > clusterStatePrimaryTerm) { synchronized (primaryTermMutex) { - if (operationPrimaryTerm > primaryTerm) { + if (opPrimaryTerm > clusterStatePrimaryTerm) { verifyNotClosed(); IndexShardState shardState = state(); @@ -2235,12 +2246,14 @@ public void acquireReplicaOperationPermit(final long operationPrimaryTerm, final synchronized (mutex) { final CountDownLatch termUpdated = new CountDownLatch(1); - if (operationPrimaryTerm > primaryTerm) { + if (opPrimaryTerm > clusterStatePrimaryTerm) { indexShardOperationPermits.asyncBlockOperations(30, TimeUnit.MINUTES, () -> { termUpdated.await(); // a primary promotion, or another primary term transition, might have been triggered concurrently to this // recheck under the operation permit if we can skip doing this work - if (operationPrimaryTerm == primaryTerm) { + if (opPrimaryTerm == clusterStatePrimaryTerm) { + assert operationPrimaryTerm < clusterStatePrimaryTerm; + operationPrimaryTerm = clusterStatePrimaryTerm; updateGlobalCheckpointOnReplica(globalCheckpoint, "primary term transition"); final long currentGlobalCheckpoint = getGlobalCheckpoint(); final long localCheckpoint; @@ -2251,7 +2264,7 @@ public void acquireReplicaOperationPermit(final long operationPrimaryTerm, final } logger.trace( "detected new primary with primary term [{}], resetting local checkpoint from [{}] to [{}]", - operationPrimaryTerm, + opPrimaryTerm, getLocalCheckpoint(), localCheckpoint); getEngine().resetLocalCheckpoint(localCheckpoint); @@ -2261,7 +2274,7 @@ public void acquireReplicaOperationPermit(final long operationPrimaryTerm, final } }, e -> failShard("exception during primary term transition", e)); - primaryTerm = operationPrimaryTerm; + clusterStatePrimaryTerm = opPrimaryTerm; termUpdated.countDown(); } } @@ -2269,20 +2282,20 @@ public void acquireReplicaOperationPermit(final long operationPrimaryTerm, final } } - assert operationPrimaryTerm <= primaryTerm - : "operation primary term [" + operationPrimaryTerm + "] should be at most [" + primaryTerm + "]"; + assert opPrimaryTerm <= clusterStatePrimaryTerm + : "operation primary term [" + opPrimaryTerm + "] should be at most [" + clusterStatePrimaryTerm + "]"; indexShardOperationPermits.acquire( new ActionListener() { @Override public void onResponse(final Releasable releasable) { - if (operationPrimaryTerm < primaryTerm) { + if (opPrimaryTerm < operationPrimaryTerm) { releasable.close(); final String message = String.format( Locale.ROOT, "%s operation primary term [%d] is too old (current [%d])", shardId, - operationPrimaryTerm, - primaryTerm); + opPrimaryTerm, + operationPrimaryTerm); onPermitAcquired.onFailure(new IllegalStateException(message)); } else { try { diff --git a/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java b/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java index e66d78f2e1a05..3b4bbe7d63530 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java +++ b/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java @@ -136,7 +136,7 @@ public void onFailure(final Exception e) { } }; - resync(shardId, indexShard.routingEntry().allocationId().getId(), indexShard.getPrimaryTerm(), wrappedSnapshot, + resync(shardId, indexShard.routingEntry().allocationId().getId(), indexShard.getOperationPrimaryTerm(), wrappedSnapshot, startingSeqNo, maxSeqNo, resyncListener); } catch (Exception e) { try { diff --git a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java index 54718c545a44e..90b082de949d3 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java @@ -394,7 +394,7 @@ private void internalRecoverFromStore(IndexShard indexShard) throws IndexShardRe final SegmentInfos segmentInfos = store.readLastCommittedSegmentsInfo(); final long maxSeqNo = Long.parseLong(segmentInfos.userData.get(SequenceNumbers.MAX_SEQ_NO)); final String translogUUID = Translog.createEmptyTranslog( - indexShard.shardPath().resolveTranslog(), maxSeqNo, shardId, indexShard.getPrimaryTerm()); + indexShard.shardPath().resolveTranslog(), maxSeqNo, shardId, indexShard.getOperationPrimaryTerm()); store.associateIndexWithNewTranslog(translogUUID); } else if (indexShouldExists) { // since we recover from local, just fill the files and size @@ -409,11 +409,12 @@ private void internalRecoverFromStore(IndexShard indexShard) throws IndexShardRe } else { store.createEmpty(); final String translogUUID = Translog.createEmptyTranslog( - indexShard.shardPath().resolveTranslog(), SequenceNumbers.NO_OPS_PERFORMED, shardId, indexShard.getPrimaryTerm()); + indexShard.shardPath().resolveTranslog(), SequenceNumbers.NO_OPS_PERFORMED, shardId, + indexShard.getOperationPrimaryTerm()); store.associateIndexWithNewTranslog(translogUUID); } indexShard.openEngineAndRecoverFromTranslog(); - indexShard.getEngine().fillSeqNoGaps(indexShard.getPrimaryTerm()); + indexShard.getEngine().fillSeqNoGaps(indexShard.getOperationPrimaryTerm()); indexShard.finalizeRecovery(); indexShard.postRecovery("post recovery from shard_store"); } catch (EngineException | IOException e) { @@ -458,11 +459,11 @@ private void restore(final IndexShard indexShard, final Repository repository, f final SegmentInfos segmentInfos = store.readLastCommittedSegmentsInfo(); final long maxSeqNo = Long.parseLong(segmentInfos.userData.get(SequenceNumbers.MAX_SEQ_NO)); final String translogUUID = Translog.createEmptyTranslog( - indexShard.shardPath().resolveTranslog(), maxSeqNo, shardId, indexShard.getPrimaryTerm()); + indexShard.shardPath().resolveTranslog(), maxSeqNo, shardId, indexShard.getOperationPrimaryTerm()); store.associateIndexWithNewTranslog(translogUUID); assert indexShard.shardRouting.primary() : "only primary shards can recover from store"; indexShard.openEngineAndRecoverFromTranslog(); - indexShard.getEngine().fillSeqNoGaps(indexShard.getPrimaryTerm()); + indexShard.getEngine().fillSeqNoGaps(indexShard.getOperationPrimaryTerm()); indexShard.finalizeRecovery(); indexShard.postRecovery("restore done"); } catch (Exception e) { diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index 57deb4666daca..c999992df57fe 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -443,7 +443,7 @@ public void cleanFiles(int totalTranslogOps, Store.MetadataSnapshot sourceMetaDa } // TODO: Assign the global checkpoint to the max_seqno of the safe commit if the index version >= 6.2 final String translogUUID = Translog.createEmptyTranslog( - indexShard.shardPath().resolveTranslog(), SequenceNumbers.UNASSIGNED_SEQ_NO, shardId, indexShard.getPrimaryTerm()); + indexShard.shardPath().resolveTranslog(), SequenceNumbers.UNASSIGNED_SEQ_NO, shardId, indexShard.getOperationPrimaryTerm()); store.associateIndexWithNewTranslog(translogUUID); } catch (CorruptIndexException | IndexFormatTooNewException | IndexFormatTooOldException ex) { // this is a fatal exception at this stage. diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java index 08301e99d6a69..5d40e2967344d 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java @@ -587,8 +587,6 @@ public void onFailure(Exception e) { public void testPrimaryReference() throws Exception { final IndexShard shard = mock(IndexShard.class); - final long primaryTerm = 1 + randomInt(200); - when(shard.getPrimaryTerm()).thenReturn(primaryTerm); AtomicBoolean closed = new AtomicBoolean(); Releasable releasable = () -> { @@ -683,7 +681,7 @@ public void testSeqNoIsSetOnPrimary() throws Exception { final IndexShard shard = mock(IndexShard.class); - when(shard.getPrimaryTerm()).thenReturn(primaryTerm); + when(shard.getClusterStatePrimaryTerm()).thenReturn(primaryTerm); when(shard.routingEntry()).thenReturn(routingEntry); when(shard.isPrimaryMode()).thenReturn(true); IndexShardRoutingTable shardRoutingTable = clusterService.state().routingTable().shardRoutingTable(shardId); @@ -1201,7 +1199,7 @@ private IndexShard mockIndexShard(ShardId shardId, ClusterService clusterService doAnswer(invocation -> { long term = (Long)invocation.getArguments()[0]; ActionListener callback = (ActionListener) invocation.getArguments()[2]; - final long primaryTerm = indexShard.getPrimaryTerm(); + final long primaryTerm = indexShard.getClusterStatePrimaryTerm(); if (term < primaryTerm) { throw new IllegalArgumentException(String.format(Locale.ROOT, "%s operation term [%d] is too old (current [%d])", shardId, term, primaryTerm)); @@ -1221,7 +1219,7 @@ private IndexShard mockIndexShard(ShardId shardId, ClusterService clusterService }); when(indexShard.isPrimaryMode()).thenAnswer(invocationOnMock -> isRelocated.get() == false); doThrow(new AssertionError("failed shard is not supported")).when(indexShard).failShard(anyString(), any(Exception.class)); - when(indexShard.getPrimaryTerm()).thenAnswer(i -> + when(indexShard.getClusterStatePrimaryTerm()).thenAnswer(i -> clusterService.state().metaData().getIndexSafe(shardId.getIndex()).primaryTerm(shardId.id())); return indexShard; } diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java index bfcc5938a8690..3be86f40897ef 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java @@ -454,7 +454,7 @@ private IndexShard mockIndexShard(ShardId shardId, ClusterService clusterService doAnswer(invocation -> { long term = (Long)invocation.getArguments()[0]; ActionListener callback = (ActionListener) invocation.getArguments()[1]; - final long primaryTerm = indexShard.getPrimaryTerm(); + final long primaryTerm = indexShard.getClusterStatePrimaryTerm(); if (term < primaryTerm) { throw new IllegalArgumentException(String.format(Locale.ROOT, "%s operation term [%d] is too old (current [%d])", shardId, term, primaryTerm)); @@ -474,7 +474,7 @@ private IndexShard mockIndexShard(ShardId shardId, ClusterService clusterService }); when(indexShard.isPrimaryMode()).thenAnswer(invocationOnMock -> isRelocated.get() == false); doThrow(new AssertionError("failed shard is not supported")).when(indexShard).failShard(anyString(), any(Exception.class)); - when(indexShard.getPrimaryTerm()).thenAnswer(i -> + when(indexShard.getClusterStatePrimaryTerm()).thenAnswer(i -> clusterService.state().metaData().getIndexSafe(shardId.getIndex()).primaryTerm(shardId.id())); return indexShard; } diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardStateIT.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardStateIT.java index aa77d7b4bf92f..5600ba054c3f7 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardStateIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardStateIT.java @@ -76,7 +76,7 @@ protected void assertPrimaryTerms(long shard0Term, long shard1Term) { if (indexService != null) { for (IndexShard shard : indexService) { assertThat("term mismatch for shard " + shard.shardId(), - shard.getPrimaryTerm(), equalTo(metaData.primaryTerm(shard.shardId().id()))); + shard.getClusterStatePrimaryTerm(), equalTo(metaData.primaryTerm(shard.shardId().id()))); } } } diff --git a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java index f299061671b7b..c5a90db3c58c3 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java @@ -240,16 +240,16 @@ public void testReplicaTermIncrementWithConcurrentPrimaryPromotion() throws Exce Collections.singletonMap("type", "{ \"type\": { \"properties\": { \"f\": { \"type\": \"keyword\"} }}}"); try (ReplicationGroup shards = new ReplicationGroup(buildIndexMetaData(2, mappings))) { shards.startAll(); - long primaryPrimaryTerm = shards.getPrimary().getPrimaryTerm(); + long primaryPrimaryTerm = shards.getPrimary().getClusterStatePrimaryTerm(); List replicas = shards.getReplicas(); IndexShard replica1 = replicas.get(0); IndexShard replica2 = replicas.get(1); shards.promoteReplicaToPrimary(replica1, (shard, listener) -> {}); - long newReplica1Term = replica1.getPrimaryTerm(); + long newReplica1Term = replica1.getClusterStatePrimaryTerm(); assertEquals(primaryPrimaryTerm + 1, newReplica1Term); - assertEquals(primaryPrimaryTerm, replica2.getPrimaryTerm()); + assertEquals(primaryPrimaryTerm, replica2.getClusterStatePrimaryTerm()); IndexRequest indexRequest = new IndexRequest(index.getName(), "type", "1").source("{ \"f\": \"1\"}", XContentType.JSON); BulkShardRequest replicationRequest = indexOnPrimary(indexRequest, replica1); @@ -278,7 +278,7 @@ public void testReplicaTermIncrementWithConcurrentPrimaryPromotion() throws Exce t1.join(); t2.join(); - assertEquals(newReplica1Term + 1, replica2.getPrimaryTerm()); + assertEquals(newReplica1Term + 1, replica2.getClusterStatePrimaryTerm()); } } @@ -303,7 +303,7 @@ protected EngineFactory getEngineFactory(ShardRouting routing) { .source("{}", XContentType.JSON) ); assertTrue(response.isFailed()); - assertNoOpTranslogOperationForDocumentFailure(shards, 1, shards.getPrimary().getPrimaryTerm(), failureMessage); + assertNoOpTranslogOperationForDocumentFailure(shards, 1, shards.getPrimary().getClusterStatePrimaryTerm(), failureMessage); shards.assertAllEqual(0); // add some replicas @@ -317,7 +317,7 @@ protected EngineFactory getEngineFactory(ShardRouting routing) { .source("{}", XContentType.JSON) ); assertTrue(response.isFailed()); - assertNoOpTranslogOperationForDocumentFailure(shards, 2, shards.getPrimary().getPrimaryTerm(), failureMessage); + assertNoOpTranslogOperationForDocumentFailure(shards, 2, shards.getPrimary().getClusterStatePrimaryTerm(), failureMessage); shards.assertAllEqual(0); } } diff --git a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java index f01d5e54a2e16..5b2862775c7cc 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java @@ -432,7 +432,8 @@ public void testResyncAfterPrimaryPromotion() throws Exception { while ((next = snapshot.next()) != null) { translogOperations++; assertThat("unexpected op: " + next, (int)next.seqNo(), lessThan(initialDocs + extraDocs)); - assertThat("unexpected primaryTerm: " + next.primaryTerm(), next.primaryTerm(), is(oldPrimary.getPrimaryTerm())); + assertThat("unexpected primaryTerm: " + next.primaryTerm(), next.primaryTerm(), + is(oldPrimary.getClusterStatePrimaryTerm())); final Translog.Source source = next.getSource(); assertThat(source.source.utf8ToString(), is("{ \"f\": \"normal\"}")); } diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 1880b6b0954dc..0d354f98216a4 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -297,7 +297,7 @@ public void testClosesPreventsNewOperations() throws InterruptedException, Execu // expected } try { - indexShard.acquireReplicaOperationPermit(indexShard.getPrimaryTerm(), SequenceNumbers.UNASSIGNED_SEQ_NO, null, + indexShard.acquireReplicaOperationPermit(indexShard.getClusterStatePrimaryTerm(), SequenceNumbers.UNASSIGNED_SEQ_NO, null, ThreadPool.Names.WRITE, ""); fail("we should not be able to increment anymore"); } catch (IndexShardClosedException e) { @@ -308,7 +308,7 @@ public void testClosesPreventsNewOperations() throws InterruptedException, Execu public void testRejectOperationPermitWithHigherTermWhenNotStarted() throws IOException { IndexShard indexShard = newShard(false); expectThrows(IndexShardNotStartedException.class, () -> - indexShard.acquireReplicaOperationPermit(indexShard.getPrimaryTerm() + randomIntBetween(1, 100), + indexShard.acquireReplicaOperationPermit(indexShard.getClusterStatePrimaryTerm() + randomIntBetween(1, 100), SequenceNumbers.UNASSIGNED_SEQ_NO, null, ThreadPool.Names.WRITE, "")); closeShards(indexShard); } @@ -331,7 +331,7 @@ public void testPrimaryPromotionDelaysOperations() throws IOException, BrokenBar throw new RuntimeException(e); } indexShard.acquireReplicaOperationPermit( - indexShard.getPrimaryTerm(), + indexShard.getClusterStatePrimaryTerm(), indexShard.getGlobalCheckpoint(), new ActionListener() { @Override @@ -427,7 +427,7 @@ public void onFailure(Exception e) { public void testPublishingOrderOnPromotion() throws IOException, InterruptedException, BrokenBarrierException { final IndexShard indexShard = newShard(false); recoveryEmptyReplica(indexShard, randomBoolean()); - final long promotedTerm = indexShard.getPrimaryTerm() + 1; + final long promotedTerm = indexShard.getClusterStatePrimaryTerm() + 1; final CyclicBarrier barrier = new CyclicBarrier(2); final AtomicBoolean stop = new AtomicBoolean(); final Thread thread = new Thread(() -> { @@ -438,7 +438,7 @@ public void testPublishingOrderOnPromotion() throws IOException, InterruptedExce } while(stop.get() == false) { if (indexShard.routingEntry().primary()) { - assertThat(indexShard.getPrimaryTerm(), equalTo(promotedTerm)); + assertThat(indexShard.getClusterStatePrimaryTerm(), equalTo(promotedTerm)); assertThat(indexShard.getReplicationGroup(), notNullValue()); } } @@ -504,7 +504,7 @@ public void testPrimaryPromotionRollsGeneration() throws Exception { // promote the replica final ShardRouting replicaRouting = indexShard.routingEntry(); - final long newPrimaryTerm = indexShard.getPrimaryTerm() + between(1, 10000); + final long newPrimaryTerm = indexShard.getClusterStatePrimaryTerm() + between(1, 10000); final ShardRouting primaryRouting = newShardRouting( replicaRouting.shardId(), @@ -558,7 +558,7 @@ public void testOperationPermitsOnPrimaryShards() throws InterruptedException, E ShardRouting replicaRouting = indexShard.routingEntry(); ShardRouting primaryRouting = newShardRouting(replicaRouting.shardId(), replicaRouting.currentNodeId(), null, true, ShardRoutingState.STARTED, replicaRouting.allocationId()); - final long newPrimaryTerm = indexShard.getPrimaryTerm() + between(1, 1000); + final long newPrimaryTerm = indexShard.getClusterStatePrimaryTerm() + between(1, 1000); indexShard.updateShardState(primaryRouting, newPrimaryTerm, (shard, listener) -> { assertThat(TestTranslog.getCurrentTerm(getTranslog(indexShard)), equalTo(newPrimaryTerm)); }, 0L, @@ -568,7 +568,7 @@ public void testOperationPermitsOnPrimaryShards() throws InterruptedException, E } else { indexShard = newStartedShard(true); } - final long primaryTerm = indexShard.getPrimaryTerm(); + final long primaryTerm = indexShard.getClusterStatePrimaryTerm(); assertEquals(0, indexShard.getActiveOperationsCount()); if (indexShard.routingEntry().isRelocationTarget() == false) { try { @@ -650,7 +650,7 @@ public void testOperationPermitOnReplicaShards() throws Exception { assertThat(e, hasToString(containsString("shard " + shardRouting + " is not a primary"))); } - final long primaryTerm = indexShard.getPrimaryTerm(); + final long primaryTerm = indexShard.getClusterStatePrimaryTerm(); final long translogGen = engineClosed ? -1 : getTranslog(indexShard).getGeneration().translogFileGeneration; final Releasable operation1; @@ -728,7 +728,7 @@ public void onFailure(Exception e) { ActionListener listener = new ActionListener() { @Override public void onResponse(Releasable releasable) { - assertThat(indexShard.getPrimaryTerm(), equalTo(newPrimaryTerm)); + assertThat(indexShard.getClusterStatePrimaryTerm(), equalTo(newPrimaryTerm)); assertThat(TestTranslog.getCurrentTerm(getTranslog(indexShard)), equalTo(newPrimaryTerm)); assertThat(indexShard.getLocalCheckpoint(), equalTo(expectedLocalCheckpoint)); assertThat(indexShard.getGlobalCheckpoint(), equalTo(newGlobalCheckPoint)); @@ -765,7 +765,7 @@ private void finish() { barrier.await(); if (indexShard.state() == IndexShardState.CREATED || indexShard.state() == IndexShardState.RECOVERING) { barrier.await(); - assertThat(indexShard.getPrimaryTerm(), equalTo(primaryTerm)); + assertThat(indexShard.getClusterStatePrimaryTerm(), equalTo(primaryTerm)); assertFalse(onResponse.get()); assertThat(onFailure.get(), instanceOf(IndexShardNotStartedException.class)); Releasables.close(operation1); @@ -774,18 +774,19 @@ private void finish() { // our operation should be blocked until the previous operations complete assertFalse(onResponse.get()); assertNull(onFailure.get()); - assertThat(indexShard.getPrimaryTerm(), equalTo(primaryTerm)); + assertThat(indexShard.getOperationPrimaryTerm(), equalTo(primaryTerm)); assertThat(TestTranslog.getCurrentTerm(getTranslog(indexShard)), equalTo(primaryTerm)); Releasables.close(operation1); // our operation should still be blocked assertFalse(onResponse.get()); assertNull(onFailure.get()); - assertThat(indexShard.getPrimaryTerm(), equalTo(primaryTerm)); + assertThat(indexShard.getOperationPrimaryTerm(), equalTo(primaryTerm)); assertThat(TestTranslog.getCurrentTerm(getTranslog(indexShard)), equalTo(primaryTerm)); Releasables.close(operation2); barrier.await(); // now lock acquisition should have succeeded - assertThat(indexShard.getPrimaryTerm(), equalTo(newPrimaryTerm)); + assertThat(indexShard.getOperationPrimaryTerm(), equalTo(newPrimaryTerm)); + assertThat(indexShard.getClusterStatePrimaryTerm(), equalTo(newPrimaryTerm)); assertThat(TestTranslog.getCurrentTerm(getTranslog(indexShard)), equalTo(newPrimaryTerm)); if (engineClosed) { assertFalse(onResponse.get()); @@ -884,7 +885,7 @@ public void testRestoreLocalCheckpointTrackerFromTranslogOnPromotion() throws IO final CountDownLatch latch = new CountDownLatch(1); indexShard.acquireReplicaOperationPermit( - indexShard.getPrimaryTerm() + 1, + indexShard.getClusterStatePrimaryTerm() + 1, globalCheckpoint, new ActionListener() { @Override @@ -906,7 +907,7 @@ public void onFailure(Exception e) { final CountDownLatch resyncLatch = new CountDownLatch(1); indexShard.updateShardState( newRouting, - indexShard.getPrimaryTerm() + 1, + indexShard.getClusterStatePrimaryTerm() + 1, (s, r) -> resyncLatch.countDown(), 1L, Collections.singleton(newRouting.allocationId().getId()), @@ -938,7 +939,7 @@ public void testThrowBackLocalCheckpointOnReplica() throws IOException, Interrup Math.toIntExact(indexShard.getLocalCheckpoint())); final CountDownLatch latch = new CountDownLatch(1); indexShard.acquireReplicaOperationPermit( - indexShard.primaryTerm + 1, + indexShard.clusterStatePrimaryTerm + 1, globalCheckpoint, new ActionListener() { @Override @@ -975,7 +976,7 @@ public void testConcurrentTermIncreaseOnReplicaShard() throws BrokenBarrierExcep final CyclicBarrier barrier = new CyclicBarrier(3); final CountDownLatch latch = new CountDownLatch(2); - final long primaryTerm = indexShard.getPrimaryTerm(); + final long primaryTerm = indexShard.getClusterStatePrimaryTerm(); final AtomicLong counter = new AtomicLong(); final AtomicReference onFailure = new AtomicReference<>(); @@ -993,7 +994,7 @@ public void testConcurrentTermIncreaseOnReplicaShard() throws BrokenBarrierExcep @Override public void onResponse(Releasable releasable) { counter.incrementAndGet(); - assertThat(indexShard.getPrimaryTerm(), equalTo(primaryTerm + increment)); + assertThat(indexShard.getClusterStatePrimaryTerm(), equalTo(primaryTerm + increment)); latch.countDown(); releasable.close(); } @@ -1037,7 +1038,7 @@ public void onFailure(Exception e) { assertThat(counter.get(), equalTo(2L)); } - assertThat(indexShard.getPrimaryTerm(), equalTo(primaryTerm + Math.max(firstIncrement, secondIncrement))); + assertThat(indexShard.getClusterStatePrimaryTerm(), equalTo(primaryTerm + Math.max(firstIncrement, secondIncrement))); closeShards(indexShard); } @@ -1719,7 +1720,7 @@ public void testRecoverFromStoreWithNoOps() throws IOException { while ((operation = snapshot.next()) != null) { if (operation.opType() == Translog.Operation.Type.NO_OP) { numNoops++; - assertEquals(newShard.getPrimaryTerm(), operation.primaryTerm()); + assertEquals(newShard.getClusterStatePrimaryTerm(), operation.primaryTerm()); assertEquals(0, operation.seqNo()); } } @@ -1826,7 +1827,7 @@ public void testRecoverFromStoreRemoveStaleOperations() throws Exception { flushShard(shard); assertThat(getShardDocUIDs(shard), containsInAnyOrder("doc-0", "doc-1")); // Simulate resync (without rollback): Noop #1, index #2 - acquireReplicaOperationPermitBlockingly(shard, shard.primaryTerm + 1); + acquireReplicaOperationPermitBlockingly(shard, shard.clusterStatePrimaryTerm + 1); shard.markSeqNoAsNoop(1, "test"); shard.applyIndexOperationOnReplica(2, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, SourceToParse.source(indexName, "_doc", "doc-2", new BytesArray("{}"), XContentType.JSON)); @@ -1837,7 +1838,8 @@ public void testRecoverFromStoreRemoveStaleOperations() throws Exception { IndexShard newShard = reinitShard(shard, newShardRouting(replicaRouting.shardId(), replicaRouting.currentNodeId(), true, ShardRoutingState.INITIALIZING, RecoverySource.StoreRecoverySource.EXISTING_STORE_INSTANCE)); - newShard.primaryTerm++; + newShard.clusterStatePrimaryTerm++; + newShard.operationPrimaryTerm++; DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); newShard.markAsRecovering("store", new RecoveryState(newShard.routingEntry(), localNode, null)); assertTrue(newShard.recoverFromStore()); @@ -2160,11 +2162,11 @@ public void testRecoverFromTranslog() throws IOException { int numCorruptEntries = 0; for (int i = 0; i < numTotalEntries; i++) { if (randomBoolean()) { - operations.add(new Translog.Index("_doc", "1", 0, primary.getPrimaryTerm(), 1, + operations.add(new Translog.Index("_doc", "1", 0, primary.getClusterStatePrimaryTerm(), 1, "{\"foo\" : \"bar\"}".getBytes(Charset.forName("UTF-8")), null, -1)); } else { // corrupt entry - operations.add(new Translog.Index("_doc", "2", 1, primary.getPrimaryTerm(), 1, + operations.add(new Translog.Index("_doc", "2", 1, primary.getClusterStatePrimaryTerm(), 1, "{\"foo\" : \"bar}".getBytes(Charset.forName("UTF-8")), null, -1)); numCorruptEntries++; } diff --git a/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java b/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java index 4444f475329b3..a34794b1b5c0f 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java @@ -83,7 +83,7 @@ public void testSyncerSendsOffCorrectDocuments() throws Exception { boolean syncNeeded = numDocs > 0; String allocationId = shard.routingEntry().allocationId().getId(); - shard.updateShardState(shard.routingEntry(), shard.getPrimaryTerm(), null, 1000L, Collections.singleton(allocationId), + shard.updateShardState(shard.routingEntry(), shard.getClusterStatePrimaryTerm(), null, 1000L, Collections.singleton(allocationId), new IndexShardRoutingTable.Builder(shard.shardId()).addShard(shard.routingEntry()).build(), Collections.emptySet()); shard.updateLocalCheckpointForShard(allocationId, globalCheckPoint); assertEquals(globalCheckPoint, shard.getGlobalCheckpoint()); @@ -142,7 +142,7 @@ public void testSyncerOnClosingShard() throws Exception { } String allocationId = shard.routingEntry().allocationId().getId(); - shard.updateShardState(shard.routingEntry(), shard.getPrimaryTerm(), null, 1000L, Collections.singleton(allocationId), + shard.updateShardState(shard.routingEntry(), shard.getClusterStatePrimaryTerm(), null, 1000L, Collections.singleton(allocationId), new IndexShardRoutingTable.Builder(shard.shardId()).addShard(shard.routingEntry()).build(), Collections.emptySet()); CountDownLatch syncCalledLatch = new CountDownLatch(1); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java index aaba17c315187..703c0c2464d99 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java @@ -201,7 +201,7 @@ public void testDifferentHistoryUUIDDisablesOPsRecovery() throws Exception { if (randomBoolean()) { // create a new translog translogUUIDtoUse = Translog.createEmptyTranslog(replica.shardPath().resolveTranslog(), flushedDocs, - replica.shardId(), replica.getPrimaryTerm()); + replica.shardId(), replica.getClusterStatePrimaryTerm()); translogGenToUse = 1; } else { translogUUIDtoUse = translogGeneration.translogUUID; diff --git a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index 026b882c98c58..6f4da3edf9012 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -234,7 +234,7 @@ public void startPrimary() throws IOException { activeIds.add(primary.routingEntry().allocationId().getId()); ShardRouting startedRoutingEntry = ShardRoutingHelper.moveToStarted(primary.routingEntry()); IndexShardRoutingTable routingTable = routingTable(shr -> shr == primary.routingEntry() ? startedRoutingEntry : shr); - primary.updateShardState(startedRoutingEntry, primary.getPrimaryTerm(), null, + primary.updateShardState(startedRoutingEntry, primary.getClusterStatePrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), activeIds, routingTable, Collections.emptySet()); for (final IndexShard replica : replicas) { recoverReplica(replica); @@ -432,7 +432,7 @@ public void syncGlobalCheckpoint() { private void updateAllocationIDsOnPrimary() throws IOException { - primary.updateShardState(primary.routingEntry(), primary.getPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), + primary.updateShardState(primary.routingEntry(), primary.getClusterStatePrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), activeIds(), routingTable(Function.identity()), Collections.emptySet()); } } @@ -534,7 +534,7 @@ public void performOn( IndexShard replica = replicationGroup.replicas.stream() .filter(s -> replicaRouting.isSameAllocation(s.routingEntry())).findFirst().get(); replica.acquireReplicaOperationPermit( - replicationGroup.primary.getPrimaryTerm(), + replicationGroup.primary.getClusterStatePrimaryTerm(), globalCheckpoint, new ActionListener() { @Override @@ -612,7 +612,7 @@ protected PrimaryResult performOnPrimary(IndexShard primary, BulkShardRequest re @Override protected void performOnReplica(BulkShardRequest request, IndexShard replica) throws Exception { - executeShardBulkOnReplica(request, replica, getPrimaryShard().getPrimaryTerm(), getPrimaryShard().getGlobalCheckpoint()); + executeShardBulkOnReplica(request, replica, getPrimaryShard().getClusterStatePrimaryTerm(), getPrimaryShard().getGlobalCheckpoint()); } } @@ -671,7 +671,7 @@ BulkShardRequest deleteOnPrimary(DeleteRequest request, IndexShard primary) thro * indexes the given requests on the supplied replica shard */ void indexOnReplica(BulkShardRequest request, ReplicationGroup group, IndexShard replica) throws Exception { - indexOnReplica(request, group, replica, group.primary.getPrimaryTerm()); + indexOnReplica(request, group, replica, group.primary.getClusterStatePrimaryTerm()); } void indexOnReplica(BulkShardRequest request, ReplicationGroup group, IndexShard replica, long term) throws Exception { @@ -682,7 +682,7 @@ void indexOnReplica(BulkShardRequest request, ReplicationGroup group, IndexShard * Executes the delete request on the given replica shard. */ void deleteOnReplica(BulkShardRequest request, ReplicationGroup group, IndexShard replica) throws Exception { - executeShardBulkOnReplica(request, replica, group.primary.getPrimaryTerm(), group.primary.getGlobalCheckpoint()); + executeShardBulkOnReplica(request, replica, group.primary.getClusterStatePrimaryTerm(), group.primary.getGlobalCheckpoint()); } class GlobalCheckpointSync extends ReplicationAction< diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index f9289f658614c..1d8f5e6dd51da 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -425,7 +425,7 @@ public static void updateRoutingEntry(IndexShard shard, ShardRouting shardRoutin IndexShardRoutingTable newRoutingTable = new IndexShardRoutingTable.Builder(shardRouting.shardId()) .addShard(shardRouting) .build(); - shard.updateShardState(shardRouting, shard.getPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), + shard.updateShardState(shardRouting, shard.getClusterStatePrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), inSyncIds, newRoutingTable, Collections.emptySet()); } @@ -514,7 +514,7 @@ protected final void recoverUnstartedReplica(final IndexShard replica, request, (int) ByteSizeUnit.MB.toBytes(1), Settings.builder().put(Node.NODE_NAME_SETTING.getKey(), pNode.getName()).build()); - primary.updateShardState(primary.routingEntry(), primary.getPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), + primary.updateShardState(primary.routingEntry(), primary.getClusterStatePrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), inSyncIds, routingTable, Collections.emptySet()); recovery.recoverToTarget(); recoveryTarget.markAsDone(); @@ -536,9 +536,9 @@ protected void startReplicaAfterRecovery(IndexShard replica, IndexShard primary, Set inSyncIdsWithReplica = new HashSet<>(inSyncIds); inSyncIdsWithReplica.add(replica.routingEntry().allocationId().getId()); // update both primary and replica shard state - primary.updateShardState(primary.routingEntry(), primary.getPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), + primary.updateShardState(primary.routingEntry(), primary.getClusterStatePrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), inSyncIdsWithReplica, newRoutingTable, Collections.emptySet()); - replica.updateShardState(replica.routingEntry().moveToStarted(), replica.getPrimaryTerm(), null, + replica.updateShardState(replica.routingEntry().moveToStarted(), replica.getClusterStatePrimaryTerm(), null, currentClusterStateVersion.get(), inSyncIdsWithReplica, newRoutingTable, Collections.emptySet()); } @@ -560,7 +560,7 @@ protected void promoteReplica(IndexShard replica, Set inSyncIds, IndexSh .removeShard(replica.routingEntry()) .addShard(routingEntry) .build(); - replica.updateShardState(routingEntry, replica.getPrimaryTerm() + 1, + replica.updateShardState(routingEntry, replica.getClusterStatePrimaryTerm() + 1, (is, listener) -> listener.onResponse(new PrimaryReplicaSyncer.ResyncTask(1, "type", "action", "desc", null, Collections.emptyMap())), currentClusterStateVersion.incrementAndGet(), From 021e83352758a960be1c18449746511504b15ecf Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 30 Jul 2018 08:55:48 +0200 Subject: [PATCH 04/17] add test + turn check into assertion --- .../index/translog/Translog.java | 5 +- .../IndexLevelReplicationTests.java | 50 +++++++++++++++++++ 2 files changed, 54 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/index/translog/Translog.java b/server/src/main/java/org/elasticsearch/index/translog/Translog.java index 31404b7874a92..04744bc68c492 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/server/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -491,7 +491,10 @@ public Location add(final Operation operation) throws IOException { try (ReleasableLock ignored = readLock.acquire()) { ensureOpen(); if (operation.primaryTerm() > current.getPrimaryTerm()) { - throw new IllegalArgumentException("Operation term is newer than the current term;" + assert false : + "Operation term is newer than the current term; " + + "current term[" + current.getPrimaryTerm() + "], operation term[" + operation + "]"; + throw new IllegalArgumentException("Operation term is newer than the current term; " + "current term[" + current.getPrimaryTerm() + "], operation term[" + operation + "]"); } return current.add(bytes, operation.seqNo()); diff --git a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java index c5a90db3c58c3..be41e800c1a9d 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java @@ -62,6 +62,7 @@ import java.util.concurrent.CyclicBarrier; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import static org.elasticsearch.index.translog.SnapshotMatchers.containsOperationsInAnyOrder; import static org.hamcrest.Matchers.anyOf; @@ -282,6 +283,55 @@ public void testReplicaTermIncrementWithConcurrentPrimaryPromotion() throws Exce } } + public void testReplicaOperationWithConcurrentPrimaryPromotion() throws Exception { + Map mappings = + Collections.singletonMap("type", "{ \"type\": { \"properties\": { \"f\": { \"type\": \"keyword\"} }}}"); + try (ReplicationGroup shards = new ReplicationGroup(buildIndexMetaData(1, mappings))) { + shards.startAll(); + long primaryPrimaryTerm = shards.getPrimary().getPrimaryTerm(); + IndexRequest indexRequest = new IndexRequest(index.getName(), "type", "1").source("{ \"f\": \"1\"}", XContentType.JSON); + BulkShardRequest replicationRequest = indexOnPrimary(indexRequest, shards.getPrimary()); + + List replicas = shards.getReplicas(); + IndexShard replica = replicas.get(0); + + CyclicBarrier barrier = new CyclicBarrier(2); + AtomicBoolean successFullyIndexed = new AtomicBoolean(); + Thread t1 = new Thread(() -> { + try { + barrier.await(); + indexOnReplica(replicationRequest, shards, replica, primaryPrimaryTerm); + successFullyIndexed.set(true); + } catch (IllegalStateException ise) { + assertThat(ise.getMessage(), containsString("is too old")); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + Thread t2 = new Thread(() -> { + try { + barrier.await(); + shards.promoteReplicaToPrimary(replica).get(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + t2.start(); + t1.start(); + t1.join(); + t2.join(); + + assertEquals(primaryPrimaryTerm + 1, replica.getPrimaryTerm()); + if (successFullyIndexed.get()) { + try(Translog.Snapshot snapshot = getTranslog(replica).newSnapshot()) { + assertThat(snapshot.totalOperations(), equalTo(1)); + Translog.Operation op = snapshot.next(); + assertThat(op.primaryTerm(), equalTo(primaryPrimaryTerm)); + } + } + } + } + /** * test document failures (failures after seq_no generation) are added as noop operation to the translog * for primary and replica shards From 78f8306f2d2795b795e060618c008519f3e20727 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 30 Jul 2018 15:49:31 +0200 Subject: [PATCH 05/17] Return term as part of operation result --- .../action/bulk/TransportShardBulkAction.java | 15 ++--- .../TransportReplicationAction.java | 2 +- .../elasticsearch/index/engine/Engine.java | 44 ++++++++----- .../index/engine/InternalEngine.java | 32 +++++---- .../elasticsearch/index/shard/IndexShard.java | 66 +++++++++++-------- .../index/shard/PrimaryReplicaSyncer.java | 2 +- .../index/shard/StoreRecovery.java | 10 +-- .../indices/recovery/RecoveryTarget.java | 3 +- .../bulk/TransportShardBulkActionTests.java | 20 +++--- .../TransportReplicationActionTests.java | 6 +- .../TransportWriteActionTests.java | 4 +- .../routing/allocation/ShardStateIT.java | 2 +- .../IndexLevelReplicationTests.java | 16 ++--- .../RecoveryDuringReplicationTests.java | 2 +- .../index/shard/IndexShardTests.java | 52 +++++++-------- .../shard/IndexingOperationListenerTests.java | 4 +- .../shard/PrimaryReplicaSyncerTests.java | 4 +- .../index/translog/TranslogTests.java | 4 +- .../recovery/RecoverySourceHandlerTests.java | 4 +- .../indices/recovery/RecoveryTests.java | 2 +- .../ESIndexLevelReplicationTestCase.java | 12 ++-- .../index/shard/IndexShardTestCase.java | 10 +-- 22 files changed, 168 insertions(+), 148 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 7e16d2d7c43b7..ed99c739afb43 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -144,7 +144,7 @@ private static BulkItemResultHolder executeIndexRequest(final IndexRequest index switch (indexResult.getResultType()) { case SUCCESS: IndexResponse response = new IndexResponse(primary.shardId(), indexRequest.type(), indexRequest.id(), - indexResult.getSeqNo(), primary.getOperationPrimaryTerm(), indexResult.getVersion(), indexResult.isCreated()); + indexResult.getSeqNo(), indexResult.getTerm(), indexResult.getVersion(), indexResult.isCreated()); return new BulkItemResultHolder(response, indexResult, bulkItemRequest); case FAILURE: return new BulkItemResultHolder(null, indexResult, bulkItemRequest); @@ -161,7 +161,7 @@ private static BulkItemResultHolder executeDeleteRequest(final DeleteRequest del switch (deleteResult.getResultType()) { case SUCCESS: DeleteResponse response = new DeleteResponse(primary.shardId(), deleteRequest.type(), deleteRequest.id(), - deleteResult.getSeqNo(), primary.getOperationPrimaryTerm(), deleteResult.getVersion(), deleteResult.isFound()); + deleteResult.getSeqNo(), deleteResult.getTerm(), deleteResult.getVersion(), deleteResult.isFound()); return new BulkItemResultHolder(response, deleteResult, bulkItemRequest); case FAILURE: return new BulkItemResultHolder(null, deleteResult, bulkItemRequest); @@ -300,7 +300,7 @@ static BulkItemResultHolder processUpdateResponse(final UpdateRequest updateRequ assert result instanceof Engine.IndexResult : result.getClass(); final IndexRequest updateIndexRequest = translate.action(); final IndexResponse indexResponse = new IndexResponse(primary.shardId(), updateIndexRequest.type(), updateIndexRequest.id(), - result.getSeqNo(), primary.getOperationPrimaryTerm(), result.getVersion(), ((Engine.IndexResult) result).isCreated()); + result.getSeqNo(), result.getTerm(), result.getVersion(), ((Engine.IndexResult) result).isCreated()); updateResponse = new UpdateResponse(indexResponse.getShardInfo(), indexResponse.getShardId(), indexResponse.getType(), indexResponse.getId(), indexResponse.getSeqNo(), indexResponse.getPrimaryTerm(), indexResponse.getVersion(), indexResponse.getResult()); @@ -320,7 +320,7 @@ static BulkItemResultHolder processUpdateResponse(final UpdateRequest updateRequ final DeleteRequest updateDeleteRequest = translate.action(); final DeleteResponse deleteResponse = new DeleteResponse(primary.shardId(), updateDeleteRequest.type(), updateDeleteRequest.id(), - result.getSeqNo(), primary.getOperationPrimaryTerm(), result.getVersion(), ((Engine.DeleteResult) result).isFound()); + result.getSeqNo(), result.getTerm(), result.getVersion(), ((Engine.DeleteResult) result).isFound()); updateResponse = new UpdateResponse(deleteResponse.getShardInfo(), deleteResponse.getShardId(), deleteResponse.getType(), deleteResponse.getId(), deleteResponse.getSeqNo(), deleteResponse.getPrimaryTerm(), @@ -356,7 +356,7 @@ static BulkItemResultHolder executeUpdateRequestOnce(UpdateRequest updateRequest } catch (Exception failure) { // we may fail translating a update to index or delete operation // we use index result to communicate failure while translating update request - final Engine.Result result = new Engine.IndexResult(failure, updateRequest.version(), SequenceNumbers.UNASSIGNED_SEQ_NO); + final Engine.Result result = primary.getFailedIndexResult(failure, updateRequest.version()); return new BulkItemResultHolder(null, result, primaryItemRequest); } @@ -490,7 +490,6 @@ public static Translog.Location performOnReplica(BulkShardRequest request, Index switch (replicaItemExecutionMode(item, i)) { case NORMAL: final DocWriteResponse primaryResponse = item.getPrimaryResponse().getResponse(); - assert replica.getOperationPrimaryTerm() == primaryResponse.getPrimaryTerm(); operationResult = performOpOnReplica(primaryResponse, docWriteRequest, replica); assert operationResult != null : "operation result must never be null when primary response has no failure"; location = syncOperationResultOrThrow(operationResult, location); @@ -560,7 +559,7 @@ static Engine.IndexResult executeIndexRequestOnPrimary(IndexRequest request, Ind () -> primary.applyIndexOperationOnPrimary(request.version(), request.versionType(), sourceToParse, request.getAutoGeneratedTimestamp(), request.isRetry()), - e -> new Engine.IndexResult(e, request.version()), + e -> primary.getFailedIndexResult(e, request.version()), mappingUpdater); } @@ -568,7 +567,7 @@ private static Engine.DeleteResult executeDeleteRequestOnPrimary(DeleteRequest r MappingUpdatePerformer mappingUpdater) throws Exception { return executeOnPrimaryWhileHandlingMappingUpdates(primary.shardId(), request.type(), () -> primary.applyDeleteOperationOnPrimary(request.version(), request.type(), request.id(), request.versionType()), - e -> new Engine.DeleteResult(e, request.version()), + e -> primary.getFailedDeleteResult(e, request.version()), mappingUpdater); } diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 5c3cfdfe08bb6..deae1b3da8837 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -929,7 +929,7 @@ private void acquirePrimaryShardReference(ShardId shardId, String allocationId, if (actualAllocationId.equals(allocationId) == false) { throw new ShardNotFoundException(shardId, "expected aID [{}] but found [{}]", allocationId, actualAllocationId); } - final long actualTerm = indexShard.getClusterStatePrimaryTerm(); + final long actualTerm = indexShard.getPendingPrimaryTerm(); if (actualTerm != primaryTerm) { throw new ShardNotFoundException(shardId, "expected aID [{}] with term [{}] but found [{}]", allocationId, primaryTerm, actualTerm); diff --git a/server/src/main/java/org/elasticsearch/index/engine/Engine.java b/server/src/main/java/org/elasticsearch/index/engine/Engine.java index b7c938b469fa6..31da7afc51a10 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -304,6 +304,7 @@ public abstract static class Result { private final Operation.TYPE operationType; private final Result.Type resultType; private final long version; + private final long term; private final long seqNo; private final Exception failure; private final SetOnce freeze = new SetOnce<>(); @@ -311,19 +312,21 @@ public abstract static class Result { private Translog.Location translogLocation; private long took; - protected Result(Operation.TYPE operationType, Exception failure, long version, long seqNo) { + protected Result(Operation.TYPE operationType, Exception failure, long version, long term, long seqNo) { this.operationType = operationType; this.failure = Objects.requireNonNull(failure); this.version = version; + this.term = term; this.seqNo = seqNo; this.requiredMappingUpdate = null; this.resultType = Type.FAILURE; } - protected Result(Operation.TYPE operationType, long version, long seqNo) { + protected Result(Operation.TYPE operationType, long version, long term, long seqNo) { this.operationType = operationType; this.version = version; this.seqNo = seqNo; + this.term = term; this.failure = null; this.requiredMappingUpdate = null; this.resultType = Type.SUCCESS; @@ -333,6 +336,7 @@ protected Result(Operation.TYPE operationType, Mapping requiredMappingUpdate) { this.operationType = operationType; this.version = Versions.NOT_FOUND; this.seqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; + this.term = 0L; this.failure = null; this.requiredMappingUpdate = requiredMappingUpdate; this.resultType = Type.MAPPING_UPDATE_REQUIRED; @@ -357,6 +361,10 @@ public long getSeqNo() { return seqNo; } + public long getTerm() { + return term; + } + /** * If the operation was aborted due to missing mappings, this method will return the mappings * that are required to complete the operation. @@ -415,20 +423,20 @@ public static class IndexResult extends Result { private final boolean created; - public IndexResult(long version, long seqNo, boolean created) { - super(Operation.TYPE.INDEX, version, seqNo); + public IndexResult(long version, long term, long seqNo, boolean created) { + super(Operation.TYPE.INDEX, version, term, seqNo); this.created = created; } /** * use in case of the index operation failed before getting to internal engine **/ - public IndexResult(Exception failure, long version) { - this(failure, version, SequenceNumbers.UNASSIGNED_SEQ_NO); + public IndexResult(Exception failure, long version, long term) { + this(failure, version, term, SequenceNumbers.UNASSIGNED_SEQ_NO); } - public IndexResult(Exception failure, long version, long seqNo) { - super(Operation.TYPE.INDEX, failure, version, seqNo); + public IndexResult(Exception failure, long version, long term, long seqNo) { + super(Operation.TYPE.INDEX, failure, version, term, seqNo); this.created = false; } @@ -447,20 +455,20 @@ public static class DeleteResult extends Result { private final boolean found; - public DeleteResult(long version, long seqNo, boolean found) { - super(Operation.TYPE.DELETE, version, seqNo); + public DeleteResult(long version, long term, long seqNo, boolean found) { + super(Operation.TYPE.DELETE, version, term, seqNo); this.found = found; } /** * use in case of the delete operation failed before getting to internal engine **/ - public DeleteResult(Exception failure, long version) { - this(failure, version, SequenceNumbers.UNASSIGNED_SEQ_NO, false); + public DeleteResult(Exception failure, long version, long term) { + this(failure, version, term, SequenceNumbers.UNASSIGNED_SEQ_NO, false); } - public DeleteResult(Exception failure, long version, long seqNo, boolean found) { - super(Operation.TYPE.DELETE, failure, version, seqNo); + public DeleteResult(Exception failure, long version, long term, long seqNo, boolean found) { + super(Operation.TYPE.DELETE, failure, version, term, seqNo); this.found = found; } @@ -477,12 +485,12 @@ public boolean isFound() { public static class NoOpResult extends Result { - NoOpResult(long seqNo) { - super(Operation.TYPE.NO_OP, 0, seqNo); + NoOpResult(long term, long seqNo) { + super(Operation.TYPE.NO_OP, term, 0, seqNo); } - NoOpResult(long seqNo, Exception failure) { - super(Operation.TYPE.NO_OP, failure, 0, seqNo); + NoOpResult(long term, long seqNo, Exception failure) { + super(Operation.TYPE.NO_OP, failure, term, 0, seqNo); } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index bdcfb2fc7313f..a30127a24ae21 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -736,6 +736,10 @@ protected long doGenerateSeqNoForOperation(final Operation operation) { return localCheckpointTracker.generateSeqNo(); } + private long getPrimaryTerm() { + return engineConfig.getPrimaryTermSupplier().getAsLong(); + } + @Override public IndexResult index(Index index) throws IOException { assert Objects.equals(index.uid().field(), IdFieldMapper.NAME) : index.uid().field(); @@ -788,7 +792,7 @@ public IndexResult index(Index index) throws IOException { indexResult = indexIntoLucene(index, plan); } else { indexResult = new IndexResult( - plan.versionForIndexing, plan.seqNoForIndexing, plan.currentNotFoundOrDeleted); + plan.versionForIndexing, getPrimaryTerm(), plan.seqNoForIndexing, plan.currentNotFoundOrDeleted); } if (index.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY) { final Translog.Location location; @@ -900,7 +904,7 @@ private IndexingStrategy planIndexingAsPrimary(Index index) throws IOException { currentVersion, index.version(), currentNotFoundOrDeleted)) { final VersionConflictEngineException e = new VersionConflictEngineException(shardId, index, currentVersion, currentNotFoundOrDeleted); - plan = IndexingStrategy.skipDueToVersionConflict(e, currentNotFoundOrDeleted, currentVersion); + plan = IndexingStrategy.skipDueToVersionConflict(e, currentNotFoundOrDeleted, currentVersion, getPrimaryTerm()); } else { plan = IndexingStrategy.processNormally(currentNotFoundOrDeleted, generateSeqNoForOperation(index), @@ -930,7 +934,7 @@ private IndexResult indexIntoLucene(Index index, IndexingStrategy plan) assert assertDocDoesNotExist(index, canOptimizeAddDocument(index) == false); addDocs(index.docs(), indexWriter); } - return new IndexResult(plan.versionForIndexing, plan.seqNoForIndexing, plan.currentNotFoundOrDeleted); + return new IndexResult(plan.versionForIndexing, getPrimaryTerm(), plan.seqNoForIndexing, plan.currentNotFoundOrDeleted); } catch (Exception ex) { if (indexWriter.getTragicException() == null) { /* There is no tragic event recorded so this must be a document failure. @@ -946,7 +950,7 @@ private IndexResult indexIntoLucene(Index index, IndexingStrategy plan) * we return a `MATCH_ANY` version to indicate no document was index. The value is * not used anyway */ - return new IndexResult(ex, Versions.MATCH_ANY, plan.seqNoForIndexing); + return new IndexResult(ex, Versions.MATCH_ANY, getPrimaryTerm(), plan.seqNoForIndexing); } else { throw ex; } @@ -1019,8 +1023,8 @@ static IndexingStrategy optimizedAppendOnly(long seqNoForIndexing) { } static IndexingStrategy skipDueToVersionConflict( - VersionConflictEngineException e, boolean currentNotFoundOrDeleted, long currentVersion) { - final IndexResult result = new IndexResult(e, currentVersion); + VersionConflictEngineException e, boolean currentNotFoundOrDeleted, long currentVersion, long term) { + final IndexResult result = new IndexResult(e, currentVersion, term); return new IndexingStrategy( currentNotFoundOrDeleted, false, false, SequenceNumbers.UNASSIGNED_SEQ_NO, Versions.NOT_FOUND, result); } @@ -1097,7 +1101,7 @@ public DeleteResult delete(Delete delete) throws IOException { deleteResult = deleteInLucene(delete, plan); } else { deleteResult = new DeleteResult( - plan.versionOfDeletion, plan.seqNoOfDeletion, plan.currentlyDeleted == false); + plan.versionOfDeletion, getPrimaryTerm(), plan.seqNoOfDeletion, plan.currentlyDeleted == false); } if (delete.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY) { final Translog.Location location; @@ -1178,7 +1182,7 @@ private DeletionStrategy planDeletionAsPrimary(Delete delete) throws IOException final DeletionStrategy plan; if (delete.versionType().isVersionConflictForWrites(currentVersion, delete.version(), currentlyDeleted)) { final VersionConflictEngineException e = new VersionConflictEngineException(shardId, delete, currentVersion, currentlyDeleted); - plan = DeletionStrategy.skipDueToVersionConflict(e, currentVersion, currentlyDeleted); + plan = DeletionStrategy.skipDueToVersionConflict(e, currentVersion, getPrimaryTerm(), currentlyDeleted); } else { plan = DeletionStrategy.processNormally( currentlyDeleted, @@ -1201,12 +1205,12 @@ private DeleteResult deleteInLucene(Delete delete, DeletionStrategy plan) new DeleteVersionValue(plan.versionOfDeletion, plan.seqNoOfDeletion, delete.primaryTerm(), engineConfig.getThreadPool().relativeTimeInMillis())); return new DeleteResult( - plan.versionOfDeletion, plan.seqNoOfDeletion, plan.currentlyDeleted == false); + plan.versionOfDeletion, getPrimaryTerm(), plan.seqNoOfDeletion, plan.currentlyDeleted == false); } catch (Exception ex) { if (indexWriter.getTragicException() == null) { // there is no tragic event and such it must be a document level failure return new DeleteResult( - ex, plan.versionOfDeletion, plan.seqNoOfDeletion, plan.currentlyDeleted == false); + ex, plan.versionOfDeletion, getPrimaryTerm(), plan.seqNoOfDeletion, plan.currentlyDeleted == false); } else { throw ex; } @@ -1237,9 +1241,9 @@ private DeletionStrategy(boolean deleteFromLucene, boolean currentlyDeleted, } static DeletionStrategy skipDueToVersionConflict( - VersionConflictEngineException e, long currentVersion, boolean currentlyDeleted) { + VersionConflictEngineException e, long currentVersion, long term, boolean currentlyDeleted) { final long unassignedSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; - final DeleteResult deleteResult = new DeleteResult(e, currentVersion, unassignedSeqNo, currentlyDeleted == false); + final DeleteResult deleteResult = new DeleteResult(e, currentVersion, term, unassignedSeqNo, currentlyDeleted == false); return new DeletionStrategy(false, currentlyDeleted, unassignedSeqNo, Versions.NOT_FOUND, deleteResult); } @@ -1268,7 +1272,7 @@ public NoOpResult noOp(final NoOp noOp) { try (ReleasableLock ignored = readLock.acquire()) { noOpResult = innerNoOp(noOp); } catch (final Exception e) { - noOpResult = new NoOpResult(noOp.seqNo(), e); + noOpResult = new NoOpResult(getPrimaryTerm(), noOp.seqNo(), e); } return noOpResult; } @@ -1278,7 +1282,7 @@ private NoOpResult innerNoOp(final NoOp noOp) throws IOException { assert noOp.seqNo() > SequenceNumbers.NO_OPS_PERFORMED; final long seqNo = noOp.seqNo(); try { - final NoOpResult noOpResult = new NoOpResult(noOp.seqNo()); + final NoOpResult noOpResult = new NoOpResult(getPrimaryTerm(), noOp.seqNo()); if (noOp.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY) { final Translog.Location location = translog.add(new Translog.NoOp(noOp.seqNo(), noOp.primaryTerm(), noOp.reason())); noOpResult.setTranslogLocation(location); diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 8d0300a3f0ffe..41fbd2b44e607 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -192,7 +192,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl protected volatile ShardRouting shardRouting; protected volatile IndexShardState state; - protected volatile long clusterStatePrimaryTerm; + protected volatile long pendingPrimaryTerm; protected volatile long operationPrimaryTerm; protected final AtomicReference currentEngineReference = new AtomicReference<>(); final EngineFactory engineFactory; @@ -316,8 +316,8 @@ public boolean shouldCache(Query query) { } indexShardOperationPermits = new IndexShardOperationPermits(shardId, threadPool); searcherWrapper = indexSearcherWrapper; - clusterStatePrimaryTerm = indexSettings.getIndexMetaData().primaryTerm(shardId.id()); - operationPrimaryTerm = clusterStatePrimaryTerm; + pendingPrimaryTerm = indexSettings.getIndexMetaData().primaryTerm(shardId.id()); + operationPrimaryTerm = pendingPrimaryTerm; refreshListeners = buildRefreshListeners(); lastSearcherAccess.set(threadPool.relativeTimeInMillis()); persistMetadata(path, indexSettings, shardRouting, null, logger); @@ -367,14 +367,14 @@ public ShardFieldData fieldData() { } /** - * Returns the primary term the index shard is on. See {@link org.elasticsearch.cluster.metadata.IndexMetaData#primaryTerm(int)} + * USE THIS METHOD WITH CARE! + * Returns the primary term the index shard is supposed to be on. In case of primary promotion or when a replica learns about + * a new term due to a new primary, the term that's exposed here will not be the term that the shard internally uses to assign + * to operations. The shard will auto-correct its internal operation term, but this might take time. + * See {@link org.elasticsearch.cluster.metadata.IndexMetaData#primaryTerm(int)} */ - public long getClusterStatePrimaryTerm() { - return this.clusterStatePrimaryTerm; - } - - public long getOperationPrimaryTerm() { - return this.operationPrimaryTerm; + public long getPendingPrimaryTerm() { + return this.pendingPrimaryTerm; } /** @@ -437,7 +437,7 @@ public void updateShardState(final ShardRouting newRouting, final CountDownLatch shardStateUpdated = new CountDownLatch(1); if (newRouting.primary()) { - if (newPrimaryTerm == clusterStatePrimaryTerm) { + if (newPrimaryTerm == pendingPrimaryTerm) { if (currentRouting.initializing() && currentRouting.isRelocationTarget() == false && newRouting.active()) { // the master started a recovering primary, activate primary mode. replicationTracker.activatePrimaryMode(getLocalCheckpoint()); @@ -460,10 +460,10 @@ public void updateShardState(final ShardRouting newRouting, assert newRouting.initializing() == false : "a started primary shard should never update its term; " + "shard " + newRouting + ", " - + "current term [" + clusterStatePrimaryTerm + "], " + + "current term [" + pendingPrimaryTerm + "], " + "new term [" + newPrimaryTerm + "]"; - assert newPrimaryTerm > clusterStatePrimaryTerm : - "primary terms can only go up; current term [" + clusterStatePrimaryTerm + "], new term [" + newPrimaryTerm + "]"; + assert newPrimaryTerm > pendingPrimaryTerm : + "primary terms can only go up; current term [" + pendingPrimaryTerm + "], new term [" + newPrimaryTerm + "]"; /* * Before this call returns, we are guaranteed that all future operations are delayed and so this happens before we * increment the primary term. The latch is needed to ensure that we do not unblock operations before the primary term is @@ -479,8 +479,8 @@ public void updateShardState(final ShardRouting newRouting, TimeUnit.MINUTES, () -> { shardStateUpdated.await(); - assert clusterStatePrimaryTerm == newPrimaryTerm : - "shard term changed on primary. expected [" + newPrimaryTerm + "] but was [" + clusterStatePrimaryTerm + "]"; + assert pendingPrimaryTerm == newPrimaryTerm : + "shard term changed on primary. expected [" + newPrimaryTerm + "] but was [" + pendingPrimaryTerm + "]"; assert operationPrimaryTerm < newPrimaryTerm; operationPrimaryTerm = newPrimaryTerm; try { @@ -530,7 +530,7 @@ public void onFailure(Exception e) { }, e -> failShard("exception during primary term transition", e)); replicationTracker.activatePrimaryMode(getLocalCheckpoint()); - clusterStatePrimaryTerm = newPrimaryTerm; + pendingPrimaryTerm = newPrimaryTerm; } } // set this last, once we finished updating all internal state. @@ -697,7 +697,7 @@ private Engine.IndexResult applyIndexOperation(long seqNo, long opPrimaryTerm, l // can not raise an exception that may block any replication of previous operations to the // replicas verifyNotClosed(e); - return new Engine.IndexResult(e, version, seqNo); + return new Engine.IndexResult(e, version, opPrimaryTerm, seqNo); } return index(getEngine(), operation); @@ -755,6 +755,14 @@ private Engine.NoOpResult noOp(Engine engine, Engine.NoOp noOp) { return engine.noOp(noOp); } + public Engine.IndexResult getFailedIndexResult(Exception e, long version) { + return new Engine.IndexResult(e, version, operationPrimaryTerm); + } + + public Engine.DeleteResult getFailedDeleteResult(Exception e, long version) { + return new Engine.DeleteResult(e, version, operationPrimaryTerm); + } + public Engine.DeleteResult applyDeleteOperationOnPrimary(long version, String type, String id, VersionType versionType) throws IOException { assert versionType.validateVersionForWrites(version); @@ -785,7 +793,7 @@ private Engine.DeleteResult applyDeleteOperation(long seqNo, long opPrimaryTerm, return new Engine.DeleteResult(update); } } catch (MapperParsingException | IllegalArgumentException | TypeMissingException e) { - return new Engine.DeleteResult(e, version, seqNo, false); + return new Engine.DeleteResult(e, version, operationPrimaryTerm, seqNo, false); } final Term uid = extractUidForDelete(type, id); final Engine.Delete delete = prepareDelete(type, id, uid, seqNo, opPrimaryTerm, version, @@ -2188,7 +2196,7 @@ private EngineConfig newEngineConfig() { IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()), Collections.singletonList(refreshListeners), Collections.singletonList(new RefreshMetricUpdater(refreshMetric)), - indexSort, this::runTranslogRecovery, circuitBreakerService, replicationTracker, this::getOperationPrimaryTerm); + indexSort, this::runTranslogRecovery, circuitBreakerService, replicationTracker, () -> operationPrimaryTerm); } /** @@ -2229,9 +2237,9 @@ public void acquireReplicaOperationPermit(final long opPrimaryTerm, final long g final Object debugInfo) { verifyNotClosed(); verifyReplicationTarget(); - if (opPrimaryTerm > clusterStatePrimaryTerm) { + if (opPrimaryTerm > pendingPrimaryTerm) { synchronized (primaryTermMutex) { - if (opPrimaryTerm > clusterStatePrimaryTerm) { + if (opPrimaryTerm > pendingPrimaryTerm) { verifyNotClosed(); IndexShardState shardState = state(); @@ -2246,14 +2254,14 @@ public void acquireReplicaOperationPermit(final long opPrimaryTerm, final long g synchronized (mutex) { final CountDownLatch termUpdated = new CountDownLatch(1); - if (opPrimaryTerm > clusterStatePrimaryTerm) { + if (opPrimaryTerm > pendingPrimaryTerm) { indexShardOperationPermits.asyncBlockOperations(30, TimeUnit.MINUTES, () -> { termUpdated.await(); // a primary promotion, or another primary term transition, might have been triggered concurrently to this // recheck under the operation permit if we can skip doing this work - if (opPrimaryTerm == clusterStatePrimaryTerm) { - assert operationPrimaryTerm < clusterStatePrimaryTerm; - operationPrimaryTerm = clusterStatePrimaryTerm; + if (opPrimaryTerm == pendingPrimaryTerm) { + assert operationPrimaryTerm < pendingPrimaryTerm; + operationPrimaryTerm = pendingPrimaryTerm; updateGlobalCheckpointOnReplica(globalCheckpoint, "primary term transition"); final long currentGlobalCheckpoint = getGlobalCheckpoint(); final long localCheckpoint; @@ -2274,7 +2282,7 @@ public void acquireReplicaOperationPermit(final long opPrimaryTerm, final long g } }, e -> failShard("exception during primary term transition", e)); - clusterStatePrimaryTerm = opPrimaryTerm; + pendingPrimaryTerm = opPrimaryTerm; termUpdated.countDown(); } } @@ -2282,8 +2290,8 @@ public void acquireReplicaOperationPermit(final long opPrimaryTerm, final long g } } - assert opPrimaryTerm <= clusterStatePrimaryTerm - : "operation primary term [" + opPrimaryTerm + "] should be at most [" + clusterStatePrimaryTerm + "]"; + assert opPrimaryTerm <= pendingPrimaryTerm + : "operation primary term [" + opPrimaryTerm + "] should be at most [" + pendingPrimaryTerm + "]"; indexShardOperationPermits.acquire( new ActionListener() { @Override diff --git a/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java b/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java index 3b4bbe7d63530..1edc0eb5dcafe 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java +++ b/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java @@ -136,7 +136,7 @@ public void onFailure(final Exception e) { } }; - resync(shardId, indexShard.routingEntry().allocationId().getId(), indexShard.getOperationPrimaryTerm(), wrappedSnapshot, + resync(shardId, indexShard.routingEntry().allocationId().getId(), indexShard.getPendingPrimaryTerm(), wrappedSnapshot, startingSeqNo, maxSeqNo, resyncListener); } catch (Exception e) { try { diff --git a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java index 90b082de949d3..e9acfe3d8b06f 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java @@ -394,7 +394,7 @@ private void internalRecoverFromStore(IndexShard indexShard) throws IndexShardRe final SegmentInfos segmentInfos = store.readLastCommittedSegmentsInfo(); final long maxSeqNo = Long.parseLong(segmentInfos.userData.get(SequenceNumbers.MAX_SEQ_NO)); final String translogUUID = Translog.createEmptyTranslog( - indexShard.shardPath().resolveTranslog(), maxSeqNo, shardId, indexShard.getOperationPrimaryTerm()); + indexShard.shardPath().resolveTranslog(), maxSeqNo, shardId, indexShard.getPendingPrimaryTerm()); store.associateIndexWithNewTranslog(translogUUID); } else if (indexShouldExists) { // since we recover from local, just fill the files and size @@ -410,11 +410,11 @@ private void internalRecoverFromStore(IndexShard indexShard) throws IndexShardRe store.createEmpty(); final String translogUUID = Translog.createEmptyTranslog( indexShard.shardPath().resolveTranslog(), SequenceNumbers.NO_OPS_PERFORMED, shardId, - indexShard.getOperationPrimaryTerm()); + indexShard.getPendingPrimaryTerm()); store.associateIndexWithNewTranslog(translogUUID); } indexShard.openEngineAndRecoverFromTranslog(); - indexShard.getEngine().fillSeqNoGaps(indexShard.getOperationPrimaryTerm()); + indexShard.getEngine().fillSeqNoGaps(indexShard.getPendingPrimaryTerm()); indexShard.finalizeRecovery(); indexShard.postRecovery("post recovery from shard_store"); } catch (EngineException | IOException e) { @@ -459,11 +459,11 @@ private void restore(final IndexShard indexShard, final Repository repository, f final SegmentInfos segmentInfos = store.readLastCommittedSegmentsInfo(); final long maxSeqNo = Long.parseLong(segmentInfos.userData.get(SequenceNumbers.MAX_SEQ_NO)); final String translogUUID = Translog.createEmptyTranslog( - indexShard.shardPath().resolveTranslog(), maxSeqNo, shardId, indexShard.getOperationPrimaryTerm()); + indexShard.shardPath().resolveTranslog(), maxSeqNo, shardId, indexShard.getPendingPrimaryTerm()); store.associateIndexWithNewTranslog(translogUUID); assert indexShard.shardRouting.primary() : "only primary shards can recover from store"; indexShard.openEngineAndRecoverFromTranslog(); - indexShard.getEngine().fillSeqNoGaps(indexShard.getOperationPrimaryTerm()); + indexShard.getEngine().fillSeqNoGaps(indexShard.getPendingPrimaryTerm()); indexShard.finalizeRecovery(); indexShard.postRecovery("restore done"); } catch (Exception e) { diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index c999992df57fe..1a772f0c3f882 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -443,7 +443,8 @@ public void cleanFiles(int totalTranslogOps, Store.MetadataSnapshot sourceMetaDa } // TODO: Assign the global checkpoint to the max_seqno of the safe commit if the index version >= 6.2 final String translogUUID = Translog.createEmptyTranslog( - indexShard.shardPath().resolveTranslog(), SequenceNumbers.UNASSIGNED_SEQ_NO, shardId, indexShard.getOperationPrimaryTerm()); + indexShard.shardPath().resolveTranslog(), SequenceNumbers.UNASSIGNED_SEQ_NO, shardId, + indexShard.getPendingPrimaryTerm()); store.associateIndexWithNewTranslog(translogUUID); } catch (CorruptIndexException | IndexFormatTooNewException | IndexFormatTooOldException ex) { // this is a fatal exception at this stage. diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportShardBulkActionTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportShardBulkActionTests.java index 006d2d04fdde4..bbe25ea02d60c 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportShardBulkActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportShardBulkActionTests.java @@ -441,7 +441,7 @@ public void testUpdateReplicaRequestWithFailure() throws Exception { BulkItemRequest replicaRequest = new BulkItemRequest(0, writeRequest); Exception err = new ElasticsearchException("I'm dead <(x.x)>"); - Engine.IndexResult indexResult = new Engine.IndexResult(err, 0, 0); + Engine.IndexResult indexResult = new Engine.IndexResult(err, 0, 0, 0); BulkItemResultHolder failedResults = new BulkItemResultHolder(null, indexResult, replicaRequest); @@ -478,7 +478,7 @@ public void testUpdateReplicaRequestWithConflictFailure() throws Exception { Exception err = new VersionConflictEngineException(shardId, "_doc", "id", "I'm conflicted <(;_;)>"); - Engine.IndexResult indexResult = new Engine.IndexResult(err, 0, 0); + Engine.IndexResult indexResult = new Engine.IndexResult(err, 0, 0, 0); BulkItemResultHolder failedResults = new BulkItemResultHolder(null, indexResult, replicaRequest); @@ -516,7 +516,7 @@ public void testUpdateReplicaRequestWithSuccess() throws Exception { boolean created = randomBoolean(); Translog.Location resultLocation = new Translog.Location(42, 42, 42); - Engine.IndexResult indexResult = new FakeResult(1, 1, created, resultLocation); + Engine.IndexResult indexResult = new FakeResult(1, 1, 1, created, resultLocation); DocWriteResponse indexResponse = new IndexResponse(shardId, "_doc", "id", 1, 17, 1, created); BulkItemResultHolder goodResults = new BulkItemResultHolder(indexResponse, indexResult, replicaRequest); @@ -559,7 +559,7 @@ public void testCalculateTranslogLocation() throws Exception { Translog.Location newLocation = new Translog.Location(1, 1, 1); final long version = randomNonNegativeLong(); final long seqNo = randomNonNegativeLong(); - Engine.IndexResult indexResult = new IndexResultWithLocation(version, seqNo, created, newLocation); + Engine.IndexResult indexResult = new IndexResultWithLocation(version, 0L, seqNo, created, newLocation); results = new BulkItemResultHolder(indexResponse, indexResult, replicaRequest); assertThat(TransportShardBulkAction.calculateTranslogLocation(original, results), equalTo(newLocation)); @@ -629,8 +629,8 @@ public void testMappingUpdateParsesCorrectNumberOfTimes() throws Exception { public class IndexResultWithLocation extends Engine.IndexResult { private final Translog.Location location; - public IndexResultWithLocation(long version, long seqNo, boolean created, Translog.Location newLocation) { - super(version, seqNo, created); + public IndexResultWithLocation(long version, long term, long seqNo, boolean created, Translog.Location newLocation) { + super(version, term, seqNo, created); this.location = newLocation; } @@ -647,8 +647,8 @@ public void testProcessUpdateResponse() throws Exception { BulkItemRequest request = new BulkItemRequest(0, updateRequest); Exception err = new VersionConflictEngineException(shardId, "_doc", "id", "I'm conflicted <(;_;)>"); - Engine.IndexResult indexResult = new Engine.IndexResult(err, 0, 0); - Engine.DeleteResult deleteResult = new Engine.DeleteResult(1, 1, true); + Engine.IndexResult indexResult = new Engine.IndexResult(err, 0, 0, 0); + Engine.DeleteResult deleteResult = new Engine.DeleteResult(1, 1, 1, true); DocWriteResponse.Result docWriteResult = DocWriteResponse.Result.CREATED; DocWriteResponse.Result deleteWriteResult = DocWriteResponse.Result.DELETED; IndexRequest indexRequest = new IndexRequest("index", "_doc", "id"); @@ -830,8 +830,8 @@ private static class FakeResult extends Engine.IndexResult { private final Translog.Location location; - protected FakeResult(long version, long seqNo, boolean created, Translog.Location location) { - super(version, seqNo, created); + protected FakeResult(long version, long term, long seqNo, boolean created, Translog.Location location) { + super(version, term, seqNo, created); this.location = location; } diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java index 5d40e2967344d..74c2896619ae8 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java @@ -681,7 +681,7 @@ public void testSeqNoIsSetOnPrimary() throws Exception { final IndexShard shard = mock(IndexShard.class); - when(shard.getClusterStatePrimaryTerm()).thenReturn(primaryTerm); + when(shard.getPendingPrimaryTerm()).thenReturn(primaryTerm); when(shard.routingEntry()).thenReturn(routingEntry); when(shard.isPrimaryMode()).thenReturn(true); IndexShardRoutingTable shardRoutingTable = clusterService.state().routingTable().shardRoutingTable(shardId); @@ -1199,7 +1199,7 @@ private IndexShard mockIndexShard(ShardId shardId, ClusterService clusterService doAnswer(invocation -> { long term = (Long)invocation.getArguments()[0]; ActionListener callback = (ActionListener) invocation.getArguments()[2]; - final long primaryTerm = indexShard.getClusterStatePrimaryTerm(); + final long primaryTerm = indexShard.getPendingPrimaryTerm(); if (term < primaryTerm) { throw new IllegalArgumentException(String.format(Locale.ROOT, "%s operation term [%d] is too old (current [%d])", shardId, term, primaryTerm)); @@ -1219,7 +1219,7 @@ private IndexShard mockIndexShard(ShardId shardId, ClusterService clusterService }); when(indexShard.isPrimaryMode()).thenAnswer(invocationOnMock -> isRelocated.get() == false); doThrow(new AssertionError("failed shard is not supported")).when(indexShard).failShard(anyString(), any(Exception.class)); - when(indexShard.getClusterStatePrimaryTerm()).thenAnswer(i -> + when(indexShard.getPendingPrimaryTerm()).thenAnswer(i -> clusterService.state().metaData().getIndexSafe(shardId.getIndex()).primaryTerm(shardId.id())); return indexShard; } diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java index 3be86f40897ef..24f50d10a1433 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java @@ -454,7 +454,7 @@ private IndexShard mockIndexShard(ShardId shardId, ClusterService clusterService doAnswer(invocation -> { long term = (Long)invocation.getArguments()[0]; ActionListener callback = (ActionListener) invocation.getArguments()[1]; - final long primaryTerm = indexShard.getClusterStatePrimaryTerm(); + final long primaryTerm = indexShard.getPendingPrimaryTerm(); if (term < primaryTerm) { throw new IllegalArgumentException(String.format(Locale.ROOT, "%s operation term [%d] is too old (current [%d])", shardId, term, primaryTerm)); @@ -474,7 +474,7 @@ private IndexShard mockIndexShard(ShardId shardId, ClusterService clusterService }); when(indexShard.isPrimaryMode()).thenAnswer(invocationOnMock -> isRelocated.get() == false); doThrow(new AssertionError("failed shard is not supported")).when(indexShard).failShard(anyString(), any(Exception.class)); - when(indexShard.getClusterStatePrimaryTerm()).thenAnswer(i -> + when(indexShard.getPendingPrimaryTerm()).thenAnswer(i -> clusterService.state().metaData().getIndexSafe(shardId.getIndex()).primaryTerm(shardId.id())); return indexShard; } diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardStateIT.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardStateIT.java index 5600ba054c3f7..530e3dd318526 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardStateIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardStateIT.java @@ -76,7 +76,7 @@ protected void assertPrimaryTerms(long shard0Term, long shard1Term) { if (indexService != null) { for (IndexShard shard : indexService) { assertThat("term mismatch for shard " + shard.shardId(), - shard.getClusterStatePrimaryTerm(), equalTo(metaData.primaryTerm(shard.shardId().id()))); + shard.getPendingPrimaryTerm(), equalTo(metaData.primaryTerm(shard.shardId().id()))); } } } diff --git a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java index be41e800c1a9d..e9a3b89af6821 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java @@ -241,16 +241,16 @@ public void testReplicaTermIncrementWithConcurrentPrimaryPromotion() throws Exce Collections.singletonMap("type", "{ \"type\": { \"properties\": { \"f\": { \"type\": \"keyword\"} }}}"); try (ReplicationGroup shards = new ReplicationGroup(buildIndexMetaData(2, mappings))) { shards.startAll(); - long primaryPrimaryTerm = shards.getPrimary().getClusterStatePrimaryTerm(); + long primaryPrimaryTerm = shards.getPrimary().getPendingPrimaryTerm(); List replicas = shards.getReplicas(); IndexShard replica1 = replicas.get(0); IndexShard replica2 = replicas.get(1); shards.promoteReplicaToPrimary(replica1, (shard, listener) -> {}); - long newReplica1Term = replica1.getClusterStatePrimaryTerm(); + long newReplica1Term = replica1.getPendingPrimaryTerm(); assertEquals(primaryPrimaryTerm + 1, newReplica1Term); - assertEquals(primaryPrimaryTerm, replica2.getClusterStatePrimaryTerm()); + assertEquals(primaryPrimaryTerm, replica2.getPendingPrimaryTerm()); IndexRequest indexRequest = new IndexRequest(index.getName(), "type", "1").source("{ \"f\": \"1\"}", XContentType.JSON); BulkShardRequest replicationRequest = indexOnPrimary(indexRequest, replica1); @@ -279,7 +279,7 @@ public void testReplicaTermIncrementWithConcurrentPrimaryPromotion() throws Exce t1.join(); t2.join(); - assertEquals(newReplica1Term + 1, replica2.getClusterStatePrimaryTerm()); + assertEquals(newReplica1Term + 1, replica2.getPendingPrimaryTerm()); } } @@ -288,7 +288,7 @@ public void testReplicaOperationWithConcurrentPrimaryPromotion() throws Exceptio Collections.singletonMap("type", "{ \"type\": { \"properties\": { \"f\": { \"type\": \"keyword\"} }}}"); try (ReplicationGroup shards = new ReplicationGroup(buildIndexMetaData(1, mappings))) { shards.startAll(); - long primaryPrimaryTerm = shards.getPrimary().getPrimaryTerm(); + long primaryPrimaryTerm = shards.getPrimary().getPendingPrimaryTerm(); IndexRequest indexRequest = new IndexRequest(index.getName(), "type", "1").source("{ \"f\": \"1\"}", XContentType.JSON); BulkShardRequest replicationRequest = indexOnPrimary(indexRequest, shards.getPrimary()); @@ -321,7 +321,7 @@ public void testReplicaOperationWithConcurrentPrimaryPromotion() throws Exceptio t1.join(); t2.join(); - assertEquals(primaryPrimaryTerm + 1, replica.getPrimaryTerm()); + assertEquals(primaryPrimaryTerm + 1, replica.getPendingPrimaryTerm()); if (successFullyIndexed.get()) { try(Translog.Snapshot snapshot = getTranslog(replica).newSnapshot()) { assertThat(snapshot.totalOperations(), equalTo(1)); @@ -353,7 +353,7 @@ protected EngineFactory getEngineFactory(ShardRouting routing) { .source("{}", XContentType.JSON) ); assertTrue(response.isFailed()); - assertNoOpTranslogOperationForDocumentFailure(shards, 1, shards.getPrimary().getClusterStatePrimaryTerm(), failureMessage); + assertNoOpTranslogOperationForDocumentFailure(shards, 1, shards.getPrimary().getPendingPrimaryTerm(), failureMessage); shards.assertAllEqual(0); // add some replicas @@ -367,7 +367,7 @@ protected EngineFactory getEngineFactory(ShardRouting routing) { .source("{}", XContentType.JSON) ); assertTrue(response.isFailed()); - assertNoOpTranslogOperationForDocumentFailure(shards, 2, shards.getPrimary().getClusterStatePrimaryTerm(), failureMessage); + assertNoOpTranslogOperationForDocumentFailure(shards, 2, shards.getPrimary().getPendingPrimaryTerm(), failureMessage); shards.assertAllEqual(0); } } diff --git a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java index 5b2862775c7cc..1e7749e003f9f 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java @@ -433,7 +433,7 @@ public void testResyncAfterPrimaryPromotion() throws Exception { translogOperations++; assertThat("unexpected op: " + next, (int)next.seqNo(), lessThan(initialDocs + extraDocs)); assertThat("unexpected primaryTerm: " + next.primaryTerm(), next.primaryTerm(), - is(oldPrimary.getClusterStatePrimaryTerm())); + is(oldPrimary.getPendingPrimaryTerm())); final Translog.Source source = next.getSource(); assertThat(source.source.utf8ToString(), is("{ \"f\": \"normal\"}")); } diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 0d354f98216a4..07f8cc39ecce8 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -297,7 +297,7 @@ public void testClosesPreventsNewOperations() throws InterruptedException, Execu // expected } try { - indexShard.acquireReplicaOperationPermit(indexShard.getClusterStatePrimaryTerm(), SequenceNumbers.UNASSIGNED_SEQ_NO, null, + indexShard.acquireReplicaOperationPermit(indexShard.getPendingPrimaryTerm(), SequenceNumbers.UNASSIGNED_SEQ_NO, null, ThreadPool.Names.WRITE, ""); fail("we should not be able to increment anymore"); } catch (IndexShardClosedException e) { @@ -308,7 +308,7 @@ public void testClosesPreventsNewOperations() throws InterruptedException, Execu public void testRejectOperationPermitWithHigherTermWhenNotStarted() throws IOException { IndexShard indexShard = newShard(false); expectThrows(IndexShardNotStartedException.class, () -> - indexShard.acquireReplicaOperationPermit(indexShard.getClusterStatePrimaryTerm() + randomIntBetween(1, 100), + indexShard.acquireReplicaOperationPermit(indexShard.getPendingPrimaryTerm() + randomIntBetween(1, 100), SequenceNumbers.UNASSIGNED_SEQ_NO, null, ThreadPool.Names.WRITE, "")); closeShards(indexShard); } @@ -331,7 +331,7 @@ public void testPrimaryPromotionDelaysOperations() throws IOException, BrokenBar throw new RuntimeException(e); } indexShard.acquireReplicaOperationPermit( - indexShard.getClusterStatePrimaryTerm(), + indexShard.getPendingPrimaryTerm(), indexShard.getGlobalCheckpoint(), new ActionListener() { @Override @@ -427,7 +427,7 @@ public void onFailure(Exception e) { public void testPublishingOrderOnPromotion() throws IOException, InterruptedException, BrokenBarrierException { final IndexShard indexShard = newShard(false); recoveryEmptyReplica(indexShard, randomBoolean()); - final long promotedTerm = indexShard.getClusterStatePrimaryTerm() + 1; + final long promotedTerm = indexShard.getPendingPrimaryTerm() + 1; final CyclicBarrier barrier = new CyclicBarrier(2); final AtomicBoolean stop = new AtomicBoolean(); final Thread thread = new Thread(() -> { @@ -438,7 +438,7 @@ public void testPublishingOrderOnPromotion() throws IOException, InterruptedExce } while(stop.get() == false) { if (indexShard.routingEntry().primary()) { - assertThat(indexShard.getClusterStatePrimaryTerm(), equalTo(promotedTerm)); + assertThat(indexShard.getPendingPrimaryTerm(), equalTo(promotedTerm)); assertThat(indexShard.getReplicationGroup(), notNullValue()); } } @@ -504,7 +504,7 @@ public void testPrimaryPromotionRollsGeneration() throws Exception { // promote the replica final ShardRouting replicaRouting = indexShard.routingEntry(); - final long newPrimaryTerm = indexShard.getClusterStatePrimaryTerm() + between(1, 10000); + final long newPrimaryTerm = indexShard.getPendingPrimaryTerm() + between(1, 10000); final ShardRouting primaryRouting = newShardRouting( replicaRouting.shardId(), @@ -558,7 +558,7 @@ public void testOperationPermitsOnPrimaryShards() throws InterruptedException, E ShardRouting replicaRouting = indexShard.routingEntry(); ShardRouting primaryRouting = newShardRouting(replicaRouting.shardId(), replicaRouting.currentNodeId(), null, true, ShardRoutingState.STARTED, replicaRouting.allocationId()); - final long newPrimaryTerm = indexShard.getClusterStatePrimaryTerm() + between(1, 1000); + final long newPrimaryTerm = indexShard.getPendingPrimaryTerm() + between(1, 1000); indexShard.updateShardState(primaryRouting, newPrimaryTerm, (shard, listener) -> { assertThat(TestTranslog.getCurrentTerm(getTranslog(indexShard)), equalTo(newPrimaryTerm)); }, 0L, @@ -568,7 +568,7 @@ public void testOperationPermitsOnPrimaryShards() throws InterruptedException, E } else { indexShard = newStartedShard(true); } - final long primaryTerm = indexShard.getClusterStatePrimaryTerm(); + final long primaryTerm = indexShard.getPendingPrimaryTerm(); assertEquals(0, indexShard.getActiveOperationsCount()); if (indexShard.routingEntry().isRelocationTarget() == false) { try { @@ -650,7 +650,7 @@ public void testOperationPermitOnReplicaShards() throws Exception { assertThat(e, hasToString(containsString("shard " + shardRouting + " is not a primary"))); } - final long primaryTerm = indexShard.getClusterStatePrimaryTerm(); + final long primaryTerm = indexShard.getPendingPrimaryTerm(); final long translogGen = engineClosed ? -1 : getTranslog(indexShard).getGeneration().translogFileGeneration; final Releasable operation1; @@ -728,7 +728,7 @@ public void onFailure(Exception e) { ActionListener listener = new ActionListener() { @Override public void onResponse(Releasable releasable) { - assertThat(indexShard.getClusterStatePrimaryTerm(), equalTo(newPrimaryTerm)); + assertThat(indexShard.getPendingPrimaryTerm(), equalTo(newPrimaryTerm)); assertThat(TestTranslog.getCurrentTerm(getTranslog(indexShard)), equalTo(newPrimaryTerm)); assertThat(indexShard.getLocalCheckpoint(), equalTo(expectedLocalCheckpoint)); assertThat(indexShard.getGlobalCheckpoint(), equalTo(newGlobalCheckPoint)); @@ -765,7 +765,7 @@ private void finish() { barrier.await(); if (indexShard.state() == IndexShardState.CREATED || indexShard.state() == IndexShardState.RECOVERING) { barrier.await(); - assertThat(indexShard.getClusterStatePrimaryTerm(), equalTo(primaryTerm)); + assertThat(indexShard.getPendingPrimaryTerm(), equalTo(primaryTerm)); assertFalse(onResponse.get()); assertThat(onFailure.get(), instanceOf(IndexShardNotStartedException.class)); Releasables.close(operation1); @@ -774,19 +774,19 @@ private void finish() { // our operation should be blocked until the previous operations complete assertFalse(onResponse.get()); assertNull(onFailure.get()); - assertThat(indexShard.getOperationPrimaryTerm(), equalTo(primaryTerm)); + assertThat(indexShard.operationPrimaryTerm, equalTo(primaryTerm)); assertThat(TestTranslog.getCurrentTerm(getTranslog(indexShard)), equalTo(primaryTerm)); Releasables.close(operation1); // our operation should still be blocked assertFalse(onResponse.get()); assertNull(onFailure.get()); - assertThat(indexShard.getOperationPrimaryTerm(), equalTo(primaryTerm)); + assertThat(indexShard.operationPrimaryTerm, equalTo(primaryTerm)); assertThat(TestTranslog.getCurrentTerm(getTranslog(indexShard)), equalTo(primaryTerm)); Releasables.close(operation2); barrier.await(); // now lock acquisition should have succeeded - assertThat(indexShard.getOperationPrimaryTerm(), equalTo(newPrimaryTerm)); - assertThat(indexShard.getClusterStatePrimaryTerm(), equalTo(newPrimaryTerm)); + assertThat(indexShard.operationPrimaryTerm, equalTo(newPrimaryTerm)); + assertThat(indexShard.getPendingPrimaryTerm(), equalTo(newPrimaryTerm)); assertThat(TestTranslog.getCurrentTerm(getTranslog(indexShard)), equalTo(newPrimaryTerm)); if (engineClosed) { assertFalse(onResponse.get()); @@ -885,7 +885,7 @@ public void testRestoreLocalCheckpointTrackerFromTranslogOnPromotion() throws IO final CountDownLatch latch = new CountDownLatch(1); indexShard.acquireReplicaOperationPermit( - indexShard.getClusterStatePrimaryTerm() + 1, + indexShard.getPendingPrimaryTerm() + 1, globalCheckpoint, new ActionListener() { @Override @@ -907,7 +907,7 @@ public void onFailure(Exception e) { final CountDownLatch resyncLatch = new CountDownLatch(1); indexShard.updateShardState( newRouting, - indexShard.getClusterStatePrimaryTerm() + 1, + indexShard.getPendingPrimaryTerm() + 1, (s, r) -> resyncLatch.countDown(), 1L, Collections.singleton(newRouting.allocationId().getId()), @@ -939,7 +939,7 @@ public void testThrowBackLocalCheckpointOnReplica() throws IOException, Interrup Math.toIntExact(indexShard.getLocalCheckpoint())); final CountDownLatch latch = new CountDownLatch(1); indexShard.acquireReplicaOperationPermit( - indexShard.clusterStatePrimaryTerm + 1, + indexShard.pendingPrimaryTerm + 1, globalCheckpoint, new ActionListener() { @Override @@ -976,7 +976,7 @@ public void testConcurrentTermIncreaseOnReplicaShard() throws BrokenBarrierExcep final CyclicBarrier barrier = new CyclicBarrier(3); final CountDownLatch latch = new CountDownLatch(2); - final long primaryTerm = indexShard.getClusterStatePrimaryTerm(); + final long primaryTerm = indexShard.getPendingPrimaryTerm(); final AtomicLong counter = new AtomicLong(); final AtomicReference onFailure = new AtomicReference<>(); @@ -994,7 +994,7 @@ public void testConcurrentTermIncreaseOnReplicaShard() throws BrokenBarrierExcep @Override public void onResponse(Releasable releasable) { counter.incrementAndGet(); - assertThat(indexShard.getClusterStatePrimaryTerm(), equalTo(primaryTerm + increment)); + assertThat(indexShard.getPendingPrimaryTerm(), equalTo(primaryTerm + increment)); latch.countDown(); releasable.close(); } @@ -1038,7 +1038,7 @@ public void onFailure(Exception e) { assertThat(counter.get(), equalTo(2L)); } - assertThat(indexShard.getClusterStatePrimaryTerm(), equalTo(primaryTerm + Math.max(firstIncrement, secondIncrement))); + assertThat(indexShard.getPendingPrimaryTerm(), equalTo(primaryTerm + Math.max(firstIncrement, secondIncrement))); closeShards(indexShard); } @@ -1720,7 +1720,7 @@ public void testRecoverFromStoreWithNoOps() throws IOException { while ((operation = snapshot.next()) != null) { if (operation.opType() == Translog.Operation.Type.NO_OP) { numNoops++; - assertEquals(newShard.getClusterStatePrimaryTerm(), operation.primaryTerm()); + assertEquals(newShard.getPendingPrimaryTerm(), operation.primaryTerm()); assertEquals(0, operation.seqNo()); } } @@ -1827,7 +1827,7 @@ public void testRecoverFromStoreRemoveStaleOperations() throws Exception { flushShard(shard); assertThat(getShardDocUIDs(shard), containsInAnyOrder("doc-0", "doc-1")); // Simulate resync (without rollback): Noop #1, index #2 - acquireReplicaOperationPermitBlockingly(shard, shard.clusterStatePrimaryTerm + 1); + acquireReplicaOperationPermitBlockingly(shard, shard.pendingPrimaryTerm + 1); shard.markSeqNoAsNoop(1, "test"); shard.applyIndexOperationOnReplica(2, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, SourceToParse.source(indexName, "_doc", "doc-2", new BytesArray("{}"), XContentType.JSON)); @@ -1838,7 +1838,7 @@ public void testRecoverFromStoreRemoveStaleOperations() throws Exception { IndexShard newShard = reinitShard(shard, newShardRouting(replicaRouting.shardId(), replicaRouting.currentNodeId(), true, ShardRoutingState.INITIALIZING, RecoverySource.StoreRecoverySource.EXISTING_STORE_INSTANCE)); - newShard.clusterStatePrimaryTerm++; + newShard.pendingPrimaryTerm++; newShard.operationPrimaryTerm++; DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); newShard.markAsRecovering("store", new RecoveryState(newShard.routingEntry(), localNode, null)); @@ -2162,11 +2162,11 @@ public void testRecoverFromTranslog() throws IOException { int numCorruptEntries = 0; for (int i = 0; i < numTotalEntries; i++) { if (randomBoolean()) { - operations.add(new Translog.Index("_doc", "1", 0, primary.getClusterStatePrimaryTerm(), 1, + operations.add(new Translog.Index("_doc", "1", 0, primary.getPendingPrimaryTerm(), 1, "{\"foo\" : \"bar\"}".getBytes(Charset.forName("UTF-8")), null, -1)); } else { // corrupt entry - operations.add(new Translog.Index("_doc", "2", 1, primary.getClusterStatePrimaryTerm(), 1, + operations.add(new Translog.Index("_doc", "2", 1, primary.getPendingPrimaryTerm(), 1, "{\"foo\" : \"bar}".getBytes(Charset.forName("UTF-8")), null, -1)); numCorruptEntries++; } diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java index 037b5d68fb865..7baae06b67151 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java @@ -148,7 +148,7 @@ public void postDelete(ShardId shardId, Engine.Delete delete, Exception ex) { ParsedDocument doc = InternalEngineTests.createParsedDoc("1", null); Engine.Delete delete = new Engine.Delete("test", "1", new Term("_id", Uid.encodeId(doc.id())), randomNonNegativeLong()); Engine.Index index = new Engine.Index(new Term("_id", Uid.encodeId(doc.id())), randomNonNegativeLong(), doc); - compositeListener.postDelete(randomShardId, delete, new Engine.DeleteResult(1, SequenceNumbers.UNASSIGNED_SEQ_NO, true)); + compositeListener.postDelete(randomShardId, delete, new Engine.DeleteResult(1, 0, SequenceNumbers.UNASSIGNED_SEQ_NO, true)); assertEquals(0, preIndex.get()); assertEquals(0, postIndex.get()); assertEquals(0, postIndexException.get()); @@ -172,7 +172,7 @@ public void postDelete(ShardId shardId, Engine.Delete delete, Exception ex) { assertEquals(2, postDelete.get()); assertEquals(2, postDeleteException.get()); - compositeListener.postIndex(randomShardId, index, new Engine.IndexResult(0, SequenceNumbers.UNASSIGNED_SEQ_NO, false)); + compositeListener.postIndex(randomShardId, index, new Engine.IndexResult(0, 0, SequenceNumbers.UNASSIGNED_SEQ_NO, false)); assertEquals(0, preIndex.get()); assertEquals(2, postIndex.get()); assertEquals(0, postIndexException.get()); diff --git a/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java b/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java index a34794b1b5c0f..ae2cc84e4870c 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java @@ -83,7 +83,7 @@ public void testSyncerSendsOffCorrectDocuments() throws Exception { boolean syncNeeded = numDocs > 0; String allocationId = shard.routingEntry().allocationId().getId(); - shard.updateShardState(shard.routingEntry(), shard.getClusterStatePrimaryTerm(), null, 1000L, Collections.singleton(allocationId), + shard.updateShardState(shard.routingEntry(), shard.getPendingPrimaryTerm(), null, 1000L, Collections.singleton(allocationId), new IndexShardRoutingTable.Builder(shard.shardId()).addShard(shard.routingEntry()).build(), Collections.emptySet()); shard.updateLocalCheckpointForShard(allocationId, globalCheckPoint); assertEquals(globalCheckPoint, shard.getGlobalCheckpoint()); @@ -142,7 +142,7 @@ public void testSyncerOnClosingShard() throws Exception { } String allocationId = shard.routingEntry().allocationId().getId(); - shard.updateShardState(shard.routingEntry(), shard.getClusterStatePrimaryTerm(), null, 1000L, Collections.singleton(allocationId), + shard.updateShardState(shard.routingEntry(), shard.getPendingPrimaryTerm(), null, 1000L, Collections.singleton(allocationId), new IndexShardRoutingTable.Builder(shard.shardId()).addShard(shard.routingEntry()).build(), Collections.emptySet()); CountDownLatch syncCalledLatch = new CountDownLatch(1); diff --git a/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index b255238c8648c..dc0d871a7f2af 100644 --- a/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -2669,7 +2669,7 @@ public void testTranslogOpSerialization() throws Exception { Engine.Index eIndex = new Engine.Index(newUid(doc), doc, randomSeqNum, randomPrimaryTerm, 1, VersionType.INTERNAL, Origin.PRIMARY, 0, 0, false); - Engine.IndexResult eIndexResult = new Engine.IndexResult(1, randomSeqNum, true); + Engine.IndexResult eIndexResult = new Engine.IndexResult(1, randomPrimaryTerm, randomSeqNum, true); Translog.Index index = new Translog.Index(eIndex, eIndexResult); BytesStreamOutput out = new BytesStreamOutput(); @@ -2680,7 +2680,7 @@ public void testTranslogOpSerialization() throws Exception { Engine.Delete eDelete = new Engine.Delete(doc.type(), doc.id(), newUid(doc), randomSeqNum, randomPrimaryTerm, 2, VersionType.INTERNAL, Origin.PRIMARY, 0); - Engine.DeleteResult eDeleteResult = new Engine.DeleteResult(2, randomSeqNum, true); + Engine.DeleteResult eDeleteResult = new Engine.DeleteResult(2, randomPrimaryTerm, randomSeqNum, true); Translog.Delete delete = new Translog.Delete(eDelete, eDeleteResult); out = new BytesStreamOutput(); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 6be6d7e80bccb..15c9109f88466 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -179,12 +179,12 @@ public void testSendSnapshotSendsOps() throws IOException { final int initialNumberOfDocs = randomIntBetween(16, 64); for (int i = 0; i < initialNumberOfDocs; i++) { final Engine.Index index = getIndex(Integer.toString(i)); - operations.add(new Translog.Index(index, new Engine.IndexResult(1, SequenceNumbers.UNASSIGNED_SEQ_NO, true))); + operations.add(new Translog.Index(index, new Engine.IndexResult(1, 1, SequenceNumbers.UNASSIGNED_SEQ_NO, true))); } final int numberOfDocsWithValidSequenceNumbers = randomIntBetween(16, 64); for (int i = initialNumberOfDocs; i < initialNumberOfDocs + numberOfDocsWithValidSequenceNumbers; i++) { final Engine.Index index = getIndex(Integer.toString(i)); - operations.add(new Translog.Index(index, new Engine.IndexResult(1, i - initialNumberOfDocs, true))); + operations.add(new Translog.Index(index, new Engine.IndexResult(1, 1, i - initialNumberOfDocs, true))); } operations.add(null); final long startingSeqNo = randomIntBetween(0, numberOfDocsWithValidSequenceNumbers - 1); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java index 703c0c2464d99..99848f6457178 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java @@ -201,7 +201,7 @@ public void testDifferentHistoryUUIDDisablesOPsRecovery() throws Exception { if (randomBoolean()) { // create a new translog translogUUIDtoUse = Translog.createEmptyTranslog(replica.shardPath().resolveTranslog(), flushedDocs, - replica.shardId(), replica.getClusterStatePrimaryTerm()); + replica.shardId(), replica.getPendingPrimaryTerm()); translogGenToUse = 1; } else { translogUUIDtoUse = translogGeneration.translogUUID; diff --git a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index 6f4da3edf9012..f186bde24385a 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -234,7 +234,7 @@ public void startPrimary() throws IOException { activeIds.add(primary.routingEntry().allocationId().getId()); ShardRouting startedRoutingEntry = ShardRoutingHelper.moveToStarted(primary.routingEntry()); IndexShardRoutingTable routingTable = routingTable(shr -> shr == primary.routingEntry() ? startedRoutingEntry : shr); - primary.updateShardState(startedRoutingEntry, primary.getClusterStatePrimaryTerm(), null, + primary.updateShardState(startedRoutingEntry, primary.getPendingPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), activeIds, routingTable, Collections.emptySet()); for (final IndexShard replica : replicas) { recoverReplica(replica); @@ -432,7 +432,7 @@ public void syncGlobalCheckpoint() { private void updateAllocationIDsOnPrimary() throws IOException { - primary.updateShardState(primary.routingEntry(), primary.getClusterStatePrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), + primary.updateShardState(primary.routingEntry(), primary.getPendingPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), activeIds(), routingTable(Function.identity()), Collections.emptySet()); } } @@ -534,7 +534,7 @@ public void performOn( IndexShard replica = replicationGroup.replicas.stream() .filter(s -> replicaRouting.isSameAllocation(s.routingEntry())).findFirst().get(); replica.acquireReplicaOperationPermit( - replicationGroup.primary.getClusterStatePrimaryTerm(), + replicationGroup.primary.getPendingPrimaryTerm(), globalCheckpoint, new ActionListener() { @Override @@ -612,7 +612,7 @@ protected PrimaryResult performOnPrimary(IndexShard primary, BulkShardRequest re @Override protected void performOnReplica(BulkShardRequest request, IndexShard replica) throws Exception { - executeShardBulkOnReplica(request, replica, getPrimaryShard().getClusterStatePrimaryTerm(), getPrimaryShard().getGlobalCheckpoint()); + executeShardBulkOnReplica(request, replica, getPrimaryShard().getPendingPrimaryTerm(), getPrimaryShard().getGlobalCheckpoint()); } } @@ -671,7 +671,7 @@ BulkShardRequest deleteOnPrimary(DeleteRequest request, IndexShard primary) thro * indexes the given requests on the supplied replica shard */ void indexOnReplica(BulkShardRequest request, ReplicationGroup group, IndexShard replica) throws Exception { - indexOnReplica(request, group, replica, group.primary.getClusterStatePrimaryTerm()); + indexOnReplica(request, group, replica, group.primary.getPendingPrimaryTerm()); } void indexOnReplica(BulkShardRequest request, ReplicationGroup group, IndexShard replica, long term) throws Exception { @@ -682,7 +682,7 @@ void indexOnReplica(BulkShardRequest request, ReplicationGroup group, IndexShard * Executes the delete request on the given replica shard. */ void deleteOnReplica(BulkShardRequest request, ReplicationGroup group, IndexShard replica) throws Exception { - executeShardBulkOnReplica(request, replica, group.primary.getClusterStatePrimaryTerm(), group.primary.getGlobalCheckpoint()); + executeShardBulkOnReplica(request, replica, group.primary.getPendingPrimaryTerm(), group.primary.getGlobalCheckpoint()); } class GlobalCheckpointSync extends ReplicationAction< diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 1d8f5e6dd51da..b2d738997f70e 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -425,7 +425,7 @@ public static void updateRoutingEntry(IndexShard shard, ShardRouting shardRoutin IndexShardRoutingTable newRoutingTable = new IndexShardRoutingTable.Builder(shardRouting.shardId()) .addShard(shardRouting) .build(); - shard.updateShardState(shardRouting, shard.getClusterStatePrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), + shard.updateShardState(shardRouting, shard.getPendingPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), inSyncIds, newRoutingTable, Collections.emptySet()); } @@ -514,7 +514,7 @@ protected final void recoverUnstartedReplica(final IndexShard replica, request, (int) ByteSizeUnit.MB.toBytes(1), Settings.builder().put(Node.NODE_NAME_SETTING.getKey(), pNode.getName()).build()); - primary.updateShardState(primary.routingEntry(), primary.getClusterStatePrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), + primary.updateShardState(primary.routingEntry(), primary.getPendingPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), inSyncIds, routingTable, Collections.emptySet()); recovery.recoverToTarget(); recoveryTarget.markAsDone(); @@ -536,9 +536,9 @@ protected void startReplicaAfterRecovery(IndexShard replica, IndexShard primary, Set inSyncIdsWithReplica = new HashSet<>(inSyncIds); inSyncIdsWithReplica.add(replica.routingEntry().allocationId().getId()); // update both primary and replica shard state - primary.updateShardState(primary.routingEntry(), primary.getClusterStatePrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), + primary.updateShardState(primary.routingEntry(), primary.getPendingPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), inSyncIdsWithReplica, newRoutingTable, Collections.emptySet()); - replica.updateShardState(replica.routingEntry().moveToStarted(), replica.getClusterStatePrimaryTerm(), null, + replica.updateShardState(replica.routingEntry().moveToStarted(), replica.getPendingPrimaryTerm(), null, currentClusterStateVersion.get(), inSyncIdsWithReplica, newRoutingTable, Collections.emptySet()); } @@ -560,7 +560,7 @@ protected void promoteReplica(IndexShard replica, Set inSyncIds, IndexSh .removeShard(replica.routingEntry()) .addShard(routingEntry) .build(); - replica.updateShardState(routingEntry, replica.getClusterStatePrimaryTerm() + 1, + replica.updateShardState(routingEntry, replica.getPendingPrimaryTerm() + 1, (is, listener) -> listener.onResponse(new PrimaryReplicaSyncer.ResyncTask(1, "type", "action", "desc", null, Collections.emptyMap())), currentClusterStateVersion.incrementAndGet(), From 70262d79b9edd779d7f978b780c5a47f98e5a805 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 30 Jul 2018 16:18:48 +0200 Subject: [PATCH 06/17] review comments --- .../elasticsearch/index/shard/IndexShard.java | 87 ++++++++++--------- 1 file changed, 44 insertions(+), 43 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 41fbd2b44e607..ddd01ecbed7c3 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -52,6 +52,7 @@ import org.elasticsearch.cluster.routing.RecoverySource.SnapshotRecoverySource; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.Booleans; +import org.elasticsearch.common.CheckedRunnable; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.stream.BytesStreamOutput; @@ -474,13 +475,11 @@ public void updateShardState(final ShardRouting newRouting, if (resyncStarted == false) { throw new IllegalStateException("cannot start resync while it's already in progress"); } - indexShardOperationPermits.asyncBlockOperations( - 30, - TimeUnit.MINUTES, + bumpPrimaryTerm(newPrimaryTerm, () -> { - shardStateUpdated.await(); assert pendingPrimaryTerm == newPrimaryTerm : - "shard term changed on primary. expected [" + newPrimaryTerm + "] but was [" + pendingPrimaryTerm + "]"; + "shard term changed on primary. expected [" + newPrimaryTerm + "] but was [" + pendingPrimaryTerm + "]" + + ", current routing: " + currentRouting + ", new routing: " + newRouting; assert operationPrimaryTerm < newPrimaryTerm; operationPrimaryTerm = newPrimaryTerm; try { @@ -527,10 +526,8 @@ public void onFailure(Exception e) { } catch (final AlreadyClosedException e) { // okay, the index was deleted } - }, - e -> failShard("exception during primary term transition", e)); + }); replicationTracker.activatePrimaryMode(getLocalCheckpoint()); - pendingPrimaryTerm = newPrimaryTerm; } } // set this last, once we finished updating all internal state. @@ -2215,7 +2212,20 @@ public void acquirePrimaryOperationPermit(ActionListener onPermitAcq indexShardOperationPermits.acquire(onPermitAcquired, executorOnDelay, false, debugInfo); } - private final Object primaryTermMutex = new Object(); + private void bumpPrimaryTerm(long newPrimaryTerm, final CheckedRunnable onBlocked) { + assert Thread.holdsLock(mutex); + assert newPrimaryTerm > pendingPrimaryTerm; + assert operationPrimaryTerm <= pendingPrimaryTerm; + final CountDownLatch termUpdated = new CountDownLatch(1); + indexShardOperationPermits.asyncBlockOperations(30, TimeUnit.MINUTES, () -> { + assert operationPrimaryTerm <= pendingPrimaryTerm; + onBlocked.run(); + assert operationPrimaryTerm <= pendingPrimaryTerm; + }, + e -> failShard("exception during primary term transition", e)); + pendingPrimaryTerm = newPrimaryTerm; + termUpdated.countDown(); + } /** * Acquire a replica operation permit whenever the shard is ready for indexing (see @@ -2238,10 +2248,8 @@ public void acquireReplicaOperationPermit(final long opPrimaryTerm, final long g verifyNotClosed(); verifyReplicationTarget(); if (opPrimaryTerm > pendingPrimaryTerm) { - synchronized (primaryTermMutex) { + synchronized (mutex) { if (opPrimaryTerm > pendingPrimaryTerm) { - verifyNotClosed(); - IndexShardState shardState = state(); // only roll translog and update primary term if shard has made it past recovery // Having a new primary term here means that the old primary failed and that there is a new primary, which again @@ -2252,39 +2260,32 @@ public void acquireReplicaOperationPermit(final long opPrimaryTerm, final long g throw new IndexShardNotStartedException(shardId, shardState); } - synchronized (mutex) { - final CountDownLatch termUpdated = new CountDownLatch(1); - if (opPrimaryTerm > pendingPrimaryTerm) { - indexShardOperationPermits.asyncBlockOperations(30, TimeUnit.MINUTES, () -> { - termUpdated.await(); - // a primary promotion, or another primary term transition, might have been triggered concurrently to this - // recheck under the operation permit if we can skip doing this work - if (opPrimaryTerm == pendingPrimaryTerm) { - assert operationPrimaryTerm < pendingPrimaryTerm; - operationPrimaryTerm = pendingPrimaryTerm; - updateGlobalCheckpointOnReplica(globalCheckpoint, "primary term transition"); - final long currentGlobalCheckpoint = getGlobalCheckpoint(); - final long localCheckpoint; - if (currentGlobalCheckpoint == SequenceNumbers.UNASSIGNED_SEQ_NO) { - localCheckpoint = SequenceNumbers.NO_OPS_PERFORMED; - } else { - localCheckpoint = currentGlobalCheckpoint; - } - logger.trace( - "detected new primary with primary term [{}], resetting local checkpoint from [{}] to [{}]", - opPrimaryTerm, - getLocalCheckpoint(), - localCheckpoint); - getEngine().resetLocalCheckpoint(localCheckpoint); - getEngine().rollTranslogGeneration(); + if (opPrimaryTerm > pendingPrimaryTerm) { + bumpPrimaryTerm(opPrimaryTerm, () -> { + // a primary promotion, or another primary term transition, might have been triggered concurrently to this + // recheck under the operation permit if we can skip doing this work + if (opPrimaryTerm == pendingPrimaryTerm) { + assert operationPrimaryTerm < pendingPrimaryTerm; + operationPrimaryTerm = pendingPrimaryTerm; + updateGlobalCheckpointOnReplica(globalCheckpoint, "primary term transition"); + final long currentGlobalCheckpoint = getGlobalCheckpoint(); + final long localCheckpoint; + if (currentGlobalCheckpoint == SequenceNumbers.UNASSIGNED_SEQ_NO) { + localCheckpoint = SequenceNumbers.NO_OPS_PERFORMED; } else { - logger.trace("a primary promotion or concurrent primary term transition has made this reset obsolete"); + localCheckpoint = currentGlobalCheckpoint; } - }, e -> failShard("exception during primary term transition", e)); - - pendingPrimaryTerm = opPrimaryTerm; - termUpdated.countDown(); - } + logger.trace( + "detected new primary with primary term [{}], resetting local checkpoint from [{}] to [{}]", + opPrimaryTerm, + getLocalCheckpoint(), + localCheckpoint); + getEngine().resetLocalCheckpoint(localCheckpoint); + getEngine().rollTranslogGeneration(); + } else { + logger.trace("a primary promotion or concurrent primary term transition has made this reset obsolete"); + } + }); } } } From afadef014c6a11e3f1a96e17ed7c16808af20639 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 30 Jul 2018 16:20:13 +0200 Subject: [PATCH 07/17] chkstyl --- .../org/elasticsearch/index/shard/IndexShardTestCase.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index b2d738997f70e..d2a84589669a6 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -514,8 +514,8 @@ protected final void recoverUnstartedReplica(final IndexShard replica, request, (int) ByteSizeUnit.MB.toBytes(1), Settings.builder().put(Node.NODE_NAME_SETTING.getKey(), pNode.getName()).build()); - primary.updateShardState(primary.routingEntry(), primary.getPendingPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), - inSyncIds, routingTable, Collections.emptySet()); + primary.updateShardState(primary.routingEntry(), primary.getPendingPrimaryTerm(), null, + currentClusterStateVersion.incrementAndGet(), inSyncIds, routingTable, Collections.emptySet()); recovery.recoverToTarget(); recoveryTarget.markAsDone(); } @@ -536,8 +536,8 @@ protected void startReplicaAfterRecovery(IndexShard replica, IndexShard primary, Set inSyncIdsWithReplica = new HashSet<>(inSyncIds); inSyncIdsWithReplica.add(replica.routingEntry().allocationId().getId()); // update both primary and replica shard state - primary.updateShardState(primary.routingEntry(), primary.getPendingPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), - inSyncIdsWithReplica, newRoutingTable, Collections.emptySet()); + primary.updateShardState(primary.routingEntry(), primary.getPendingPrimaryTerm(), null, + currentClusterStateVersion.incrementAndGet(), inSyncIdsWithReplica, newRoutingTable, Collections.emptySet()); replica.updateShardState(replica.routingEntry().moveToStarted(), replica.getPendingPrimaryTerm(), null, currentClusterStateVersion.get(), inSyncIdsWithReplica, newRoutingTable, Collections.emptySet()); } From 0a8ab47f4e758f86bb678ba21f1589584ac9f30c Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 30 Jul 2018 16:49:32 +0200 Subject: [PATCH 08/17] more ckstyl --- .../index/replication/ESIndexLevelReplicationTestCase.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index f186bde24385a..a40f950b02e0d 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -432,7 +432,8 @@ public void syncGlobalCheckpoint() { private void updateAllocationIDsOnPrimary() throws IOException { - primary.updateShardState(primary.routingEntry(), primary.getPendingPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), + primary.updateShardState(primary.routingEntry(), primary.getPendingPrimaryTerm(), null, + currentClusterStateVersion.incrementAndGet(), activeIds(), routingTable(Function.identity()), Collections.emptySet()); } } From 78a7e820d85faa7d65ba2b0fe72613dcd17e2903 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Tue, 31 Jul 2018 11:45:57 +0200 Subject: [PATCH 09/17] oh my --- .../src/main/java/org/elasticsearch/index/shard/IndexShard.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index ddd01ecbed7c3..0d4cae42e9572 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -2219,6 +2219,7 @@ private void bumpPrimaryTerm(long newPrimaryTerm, final Ch final CountDownLatch termUpdated = new CountDownLatch(1); indexShardOperationPermits.asyncBlockOperations(30, TimeUnit.MINUTES, () -> { assert operationPrimaryTerm <= pendingPrimaryTerm; + termUpdated.await(); onBlocked.run(); assert operationPrimaryTerm <= pendingPrimaryTerm; }, From 63314f46ffb65e9a115e4988eefe99163460852b Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Tue, 31 Jul 2018 13:46:56 +0200 Subject: [PATCH 10/17] still need for shard state to be updated (in particular for primary mode to be activated on the replication tracker) --- .../src/main/java/org/elasticsearch/index/shard/IndexShard.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 0d4cae42e9572..475ca8a4cc0d1 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -477,6 +477,7 @@ public void updateShardState(final ShardRouting newRouting, } bumpPrimaryTerm(newPrimaryTerm, () -> { + shardStateUpdated.await(); assert pendingPrimaryTerm == newPrimaryTerm : "shard term changed on primary. expected [" + newPrimaryTerm + "] but was [" + pendingPrimaryTerm + "]" + ", current routing: " + currentRouting + ", new routing: " + newRouting; From f7c4ae0db5c22ca4e8d020ff4c9d716ccda99ad9 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Tue, 31 Jul 2018 13:52:26 +0200 Subject: [PATCH 11/17] activate primary mode under operation block --- .../src/main/java/org/elasticsearch/index/shard/IndexShard.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 475ca8a4cc0d1..3c43173c96135 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -484,6 +484,7 @@ public void updateShardState(final ShardRouting newRouting, assert operationPrimaryTerm < newPrimaryTerm; operationPrimaryTerm = newPrimaryTerm; try { + replicationTracker.activatePrimaryMode(getLocalCheckpoint()); /* * If this shard was serving as a replica shard when another shard was promoted to primary then the state of * its local checkpoint tracker was reset during the primary term transition. In particular, the local @@ -528,7 +529,6 @@ public void onFailure(Exception e) { // okay, the index was deleted } }); - replicationTracker.activatePrimaryMode(getLocalCheckpoint()); } } // set this last, once we finished updating all internal state. From 42d59d8f08d615b86f7b61be6ffd2fd1a635b8fe Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Tue, 31 Jul 2018 21:43:57 +0200 Subject: [PATCH 12/17] attempt at fixing yet another issue --- .../elasticsearch/index/shard/IndexShard.java | 20 +++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 3c43173c96135..d412987fb95b7 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -425,11 +425,18 @@ public void updateShardState(final ShardRouting newRouting, "a primary relocation is completed by the master, but primary mode is not active " + currentRouting; changeState(IndexShardState.STARTED, "global state is [" + newRouting.state() + "]"); - } else if (currentRouting.primary() && currentRouting.relocating() && replicationTracker.isPrimaryMode() == false && + } else if (currentRouting.primary() && currentRouting.relocating() && + operationPrimaryTerm == pendingPrimaryTerm && + replicationTracker.isPrimaryMode() == false && (newRouting.relocating() == false || newRouting.equalsIgnoringMetaData(currentRouting) == false)) { // if the shard is not in primary mode anymore (after primary relocation) we have to fail when any changes in shard routing occur (e.g. due to recovery // failure / cancellation). The reason is that at the moment we cannot safely reactivate primary mode without risking two // active primaries. + // We check for operationPrimaryTerm to be equal to pendingPrimaryTerm, which ensures that we have a fully baked primary, + // i.e. a primary that has has transitioned to primary mode. Assume for example an active replica, which then got a + // cluster state where it became promoted to relocating primary. This means that we asynchronously start the transition to + // primary in the background. A follow-up cluster state might then cancel relocation before we have completed the transition + // to primary, which would result in the shard to be failed if we did not check for the operationPrimaryTerm. throw new IndexShardRelocatedException(shardId(), "Shard is marked as relocated, cannot safely move to state " + newRouting.state()); } assert newRouting.active() == false || state == IndexShardState.STARTED || state == IndexShardState.CLOSED : @@ -482,9 +489,13 @@ public void updateShardState(final ShardRouting newRouting, "shard term changed on primary. expected [" + newPrimaryTerm + "] but was [" + pendingPrimaryTerm + "]" + ", current routing: " + currentRouting + ", new routing: " + newRouting; assert operationPrimaryTerm < newPrimaryTerm; - operationPrimaryTerm = newPrimaryTerm; try { - replicationTracker.activatePrimaryMode(getLocalCheckpoint()); + synchronized (mutex) { + assert currentRouting.primary(); + // do these updates under the mutex as this otherwise races with subsequent calls of updateShardState + operationPrimaryTerm = newPrimaryTerm; + replicationTracker.activatePrimaryMode(getLocalCheckpoint()); + } /* * If this shard was serving as a replica shard when another shard was promoted to primary then the state of * its local checkpoint tracker was reset during the primary term transition. In particular, the local @@ -536,8 +547,9 @@ public void onFailure(Exception e) { assert this.shardRouting.primary() == false || this.shardRouting.started() == false || // note that we use started and not active to avoid relocating shards + operationPrimaryTerm != pendingPrimaryTerm || this.replicationTracker.isPrimaryMode() - : "an started primary must be in primary mode " + this.shardRouting; + : "a started primary with non-pending operation term must be in primary mode " + this.shardRouting; shardStateUpdated.countDown(); } if (currentRouting != null && currentRouting.active() == false && newRouting.active()) { From 9a4256286c995a8d887225bbd2ddae6a17108786 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Tue, 31 Jul 2018 23:00:04 +0200 Subject: [PATCH 13/17] fix assertion + tests --- .../main/java/org/elasticsearch/index/shard/IndexShard.java | 2 +- .../index/replication/IndexLevelReplicationTests.java | 4 ++-- .../index/replication/RecoveryDuringReplicationTests.java | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index d412987fb95b7..499ee19df7ab9 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -491,7 +491,7 @@ public void updateShardState(final ShardRouting newRouting, assert operationPrimaryTerm < newPrimaryTerm; try { synchronized (mutex) { - assert currentRouting.primary(); + assert shardRouting.primary(); // do these updates under the mutex as this otherwise races with subsequent calls of updateShardState operationPrimaryTerm = newPrimaryTerm; replicationTracker.activatePrimaryMode(getLocalCheckpoint()); diff --git a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java index e9a3b89af6821..1e2c00e5896ff 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java @@ -223,7 +223,7 @@ public void testConflictingOpsOnReplica() throws Exception { } logger.info("--> promoting replica to primary " + replica1.routingEntry()); - shards.promoteReplicaToPrimary(replica1); + shards.promoteReplicaToPrimary(replica1).get(); indexRequest = new IndexRequest(index.getName(), "type", "1").source("{ \"f\": \"2\"}", XContentType.JSON); shards.index(indexRequest); shards.refresh("test"); @@ -459,7 +459,7 @@ public void testSeqNoCollision() throws Exception { // Make sure that peer-recovery transfers all but non-overridden operations. IndexShard replica3 = shards.addReplica(); logger.info("--> Promote replica2 as the primary"); - shards.promoteReplicaToPrimary(replica2); + shards.promoteReplicaToPrimary(replica2).get(); logger.info("--> Recover replica3 from replica2"); recoverReplica(replica3, replica2, true); try (Translog.Snapshot snapshot = getTranslog(replica3).newSnapshot()) { diff --git a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java index 1e7749e003f9f..2d198c32ba74f 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java @@ -245,7 +245,7 @@ public void testRecoveryAfterPrimaryPromotion() throws Exception { } } - shards.promoteReplicaToPrimary(newPrimary); + shards.promoteReplicaToPrimary(newPrimary).get(); // check that local checkpoint of new primary is properly tracked after primary promotion assertThat(newPrimary.getLocalCheckpoint(), equalTo(totalDocs - 1L)); From ceb330b028f05369d26b425f1f0fd9af5fd641df Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Tue, 31 Jul 2018 23:53:37 +0200 Subject: [PATCH 14/17] more test fixes --- .../index/shard/IndexShardTests.java | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index b4028e235ee19..22c7a7469a4d9 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -418,11 +418,8 @@ public void onFailure(Exception e) { } /** - * This test makes sure that people can use the shard routing entry to check whether a shard was already promoted to - * a primary. Concretely this means, that when we publish the routing entry via {@link IndexShard#routingEntry()} the following - * should have happened - * 1) Internal state (ala ReplicationTracker) have been updated - * 2) Primary term is set to the new term + * This test makes sure that people can use the shard routing entry + take an operation permit to check whether a shard was already + * promoted to a primary. */ public void testPublishingOrderOnPromotion() throws IOException, InterruptedException, BrokenBarrierException { final IndexShard indexShard = newShard(false); @@ -439,7 +436,11 @@ public void testPublishingOrderOnPromotion() throws IOException, InterruptedExce while(stop.get() == false) { if (indexShard.routingEntry().primary()) { assertThat(indexShard.getPendingPrimaryTerm(), equalTo(promotedTerm)); - assertThat(indexShard.getReplicationGroup(), notNullValue()); + final PlainActionFuture permitAcquiredFuture = new PlainActionFuture<>(); + indexShard.acquirePrimaryOperationPermit(permitAcquiredFuture, ThreadPool.Names.SAME, "bla"); + try (Releasable ignored = permitAcquiredFuture.actionGet()) { + assertThat(indexShard.getReplicationGroup(), notNullValue()); + } } } }); @@ -572,7 +573,10 @@ public void testOperationPermitsOnPrimaryShards() throws InterruptedException, E assertEquals(0, indexShard.getActiveOperationsCount()); if (indexShard.routingEntry().isRelocationTarget() == false) { try { - indexShard.acquireReplicaOperationPermit(primaryTerm, indexShard.getGlobalCheckpoint(), null, ThreadPool.Names.WRITE, ""); + final PlainActionFuture permitAcquiredFuture = new PlainActionFuture<>(); + indexShard.acquireReplicaOperationPermit(primaryTerm, indexShard.getGlobalCheckpoint(), permitAcquiredFuture, + ThreadPool.Names.WRITE, ""); + permitAcquiredFuture.actionGet(); fail("shard shouldn't accept operations as replica"); } catch (IllegalStateException ignored) { From c328417933e370a6f4889f661b1079a6202d9ba2 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 1 Aug 2018 09:06:13 +0200 Subject: [PATCH 15/17] add isRelocated flag to simplify code reasoning --- .../TransportReplicationAction.java | 2 +- .../index/seqno/ReplicationTracker.java | 18 ++++++++++++++++++ .../elasticsearch/index/shard/IndexShard.java | 18 ++++++------------ .../recovery/RecoverySourceHandler.java | 2 +- .../TransportReplicationActionTests.java | 4 ++-- .../replication/TransportWriteActionTests.java | 2 +- .../index/seqno/ReplicationTrackerTests.java | 2 ++ .../index/shard/IndexShardTests.java | 16 ++++++++-------- .../recovery/RecoverySourceHandlerTests.java | 4 ++-- 9 files changed, 41 insertions(+), 27 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index deae1b3da8837..dbdd5acae1fc6 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -983,7 +983,7 @@ class PrimaryShardReference extends ShardReference } public boolean isRelocated() { - return indexShard.isPrimaryMode() == false; + return indexShard.isRelocatedPrimary(); } @Override diff --git a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java index 6548aad767006..e868da5e82ac6 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java @@ -85,6 +85,7 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L * computation from that point on. */ volatile boolean primaryMode; + /** * Boolean flag that indicates if a relocation handoff is in progress. A handoff is started by calling {@link #startRelocationHandoff} * and is finished by either calling {@link #completeRelocationHandoff} or {@link #abortRelocationHandoff}, depending on whether the @@ -102,6 +103,11 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L */ boolean handoffInProgress; + /** + * Boolean flag that indicates whether a relocation handoff completed (see {@link #completeRelocationHandoff}). + */ + volatile boolean relocated; + /** * The global checkpoint tracker relies on the property that cluster state updates are applied in-order. After transferring a primary * context from the primary relocation source to the target and initializing the target, it is possible for the target to apply a @@ -260,6 +266,13 @@ public boolean isPrimaryMode() { return primaryMode; } + /** + * Returns whether the replication tracker has relocated away to another shard copy. + */ + public boolean isRelocated() { + return relocated; + } + /** * Class invariant that should hold before and after every invocation of public methods on this class. As Java lacks implication * as a logical operator, many of the invariants are written under the form (!A || B), they should be read as (A implies B) however. @@ -287,6 +300,9 @@ private boolean invariant() { // relocation handoff can only occur in primary mode assert !handoffInProgress || primaryMode; + // a relocated copy is not in primary mode + assert !relocated || !primaryMode; + // the current shard is marked as in-sync when the global checkpoint tracker operates in primary mode assert !primaryMode || checkpoints.get(shardAllocationId).inSync; @@ -766,8 +782,10 @@ public synchronized void completeRelocationHandoff() { assert invariant(); assert primaryMode; assert handoffInProgress; + assert relocated == false; primaryMode = false; handoffInProgress = false; + relocated = true; // forget all checkpoint information except for global checkpoint of current shard checkpoints.entrySet().stream().forEach(e -> { final CheckpointState cps = e.getValue(); diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 499ee19df7ab9..e78c5c9eb08d1 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -425,18 +425,11 @@ public void updateShardState(final ShardRouting newRouting, "a primary relocation is completed by the master, but primary mode is not active " + currentRouting; changeState(IndexShardState.STARTED, "global state is [" + newRouting.state() + "]"); - } else if (currentRouting.primary() && currentRouting.relocating() && - operationPrimaryTerm == pendingPrimaryTerm && - replicationTracker.isPrimaryMode() == false && + } else if (currentRouting.primary() && currentRouting.relocating() && replicationTracker.isRelocated() && (newRouting.relocating() == false || newRouting.equalsIgnoringMetaData(currentRouting) == false)) { // if the shard is not in primary mode anymore (after primary relocation) we have to fail when any changes in shard routing occur (e.g. due to recovery // failure / cancellation). The reason is that at the moment we cannot safely reactivate primary mode without risking two // active primaries. - // We check for operationPrimaryTerm to be equal to pendingPrimaryTerm, which ensures that we have a fully baked primary, - // i.e. a primary that has has transitioned to primary mode. Assume for example an active replica, which then got a - // cluster state where it became promoted to relocating primary. This means that we asynchronously start the transition to - // primary in the background. A follow-up cluster state might then cancel relocation before we have completed the transition - // to primary, which would result in the shard to be failed if we did not check for the operationPrimaryTerm. throw new IndexShardRelocatedException(shardId(), "Shard is marked as relocated, cannot safely move to state " + newRouting.state()); } assert newRouting.active() == false || state == IndexShardState.STARTED || state == IndexShardState.CLOSED : @@ -610,7 +603,7 @@ public void relocated(final Consumer consumer consumer.accept(primaryContext); synchronized (mutex) { verifyRelocatingState(); - replicationTracker.completeRelocationHandoff(); // make changes to primaryMode flag only under mutex + replicationTracker.completeRelocationHandoff(); // make changes to primaryMode and relocated flag only under mutex } } catch (final Exception e) { try { @@ -2113,10 +2106,11 @@ public void startRecovery(RecoveryState recoveryState, PeerRecoveryTargetService } /** - * Returns whether the shard is in primary mode, i.e., in charge of replicating changes (see {@link ReplicationTracker}). + * Returns whether the shard is a relocated primary, i.e. not in charge anymore of replicating changes (see {@link ReplicationTracker}). */ - public boolean isPrimaryMode() { - return replicationTracker.isPrimaryMode(); + public boolean isRelocatedPrimary() { + assert shardRouting.primary() : "only call isRelocatedPrimary on primary shard"; + return replicationTracker.isRelocated(); } class ShardEventListener implements Engine.EventListener { diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 45500349865f7..352f07d57649d 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -250,7 +250,7 @@ public void onFailure(Exception e) { try (Releasable ignored = FutureUtils.get(permit)) { // check that the IndexShard still has the primary authority. This needs to be checked under operation permit to prevent // races, as IndexShard will switch its authority only when it holds all operation permits, see IndexShard.relocated() - if (primary.isPrimaryMode() == false) { + if (primary.isRelocatedPrimary()) { throw new IndexShardRelocatedException(primary.shardId()); } runnable.run(); diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java index 74c2896619ae8..6756c00ea84aa 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java @@ -683,7 +683,7 @@ public void testSeqNoIsSetOnPrimary() throws Exception { final IndexShard shard = mock(IndexShard.class); when(shard.getPendingPrimaryTerm()).thenReturn(primaryTerm); when(shard.routingEntry()).thenReturn(routingEntry); - when(shard.isPrimaryMode()).thenReturn(true); + when(shard.isRelocatedPrimary()).thenReturn(false); IndexShardRoutingTable shardRoutingTable = clusterService.state().routingTable().shardRoutingTable(shardId); Set inSyncIds = randomBoolean() ? Collections.singleton(routingEntry.allocationId().getId()) : clusterService.state().metaData().index(index).inSyncAllocationIds(0); @@ -1217,7 +1217,7 @@ private IndexShard mockIndexShard(ShardId shardId, ClusterService clusterService } return routing; }); - when(indexShard.isPrimaryMode()).thenAnswer(invocationOnMock -> isRelocated.get() == false); + when(indexShard.isRelocatedPrimary()).thenAnswer(invocationOnMock -> isRelocated.get()); doThrow(new AssertionError("failed shard is not supported")).when(indexShard).failShard(anyString(), any(Exception.class)); when(indexShard.getPendingPrimaryTerm()).thenAnswer(i -> clusterService.state().metaData().getIndexSafe(shardId.getIndex()).primaryTerm(shardId.id())); diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java index 24f50d10a1433..d305630f83ed7 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java @@ -472,7 +472,7 @@ private IndexShard mockIndexShard(ShardId shardId, ClusterService clusterService } return routing; }); - when(indexShard.isPrimaryMode()).thenAnswer(invocationOnMock -> isRelocated.get() == false); + when(indexShard.isRelocatedPrimary()).thenAnswer(invocationOnMock -> isRelocated.get()); doThrow(new AssertionError("failed shard is not supported")).when(indexShard).failShard(anyString(), any(Exception.class)); when(indexShard.getPendingPrimaryTerm()).thenAnswer(i -> clusterService.state().metaData().getIndexSafe(shardId.getIndex()).primaryTerm(shardId.id())); diff --git a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTests.java b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTests.java index 6fdce76912e04..e001f82809b07 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTests.java @@ -770,8 +770,10 @@ public void testPrimaryContextHandoff() throws IOException { assertThat(newPrimary.routingTable, equalTo(oldPrimary.routingTable)); assertThat(newPrimary.replicationGroup, equalTo(oldPrimary.replicationGroup)); + assertFalse(oldPrimary.relocated); oldPrimary.completeRelocationHandoff(); assertFalse(oldPrimary.primaryMode); + assertTrue(oldPrimary.relocated); } public void testIllegalStateExceptionIfUnknownAllocationId() { diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 22c7a7469a4d9..03442be7f0662 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -1421,14 +1421,14 @@ public void testLockingBeforeAndAfterRelocated() throws Exception { recoveryThread.start(); latch.await(); // recovery can only be finalized after we release the current primaryOperationLock - assertTrue(shard.isPrimaryMode()); + assertFalse(shard.isRelocatedPrimary()); } // recovery can be now finalized recoveryThread.join(); - assertFalse(shard.isPrimaryMode()); + assertTrue(shard.isRelocatedPrimary()); try (Releasable ignored = acquirePrimaryOperationPermitBlockingly(shard)) { // lock can again be acquired - assertFalse(shard.isPrimaryMode()); + assertTrue(shard.isRelocatedPrimary()); } closeShards(shard); @@ -1470,7 +1470,7 @@ public void onResponse(Releasable releasable) { public void testStressRelocated() throws Exception { final IndexShard shard = newStartedShard(true); - assertTrue(shard.isPrimaryMode()); + assertFalse(shard.isRelocatedPrimary()); IndexShardTestCase.updateRoutingEntry(shard, ShardRoutingHelper.relocate(shard.routingEntry(), "other_node")); final int numThreads = randomIntBetween(2, 4); Thread[] indexThreads = new Thread[numThreads]; @@ -1506,14 +1506,14 @@ public void run() { assertThat(relocated.get(), equalTo(false)); assertThat(shard.getActiveOperationsCount(), greaterThan(0)); // ensure we only transition after pending operations completed - assertTrue(shard.isPrimaryMode()); + assertFalse(shard.isRelocatedPrimary()); // complete pending operations barrier.await(); // complete recovery/relocation recoveryThread.join(); // ensure relocated successfully once pending operations are done assertThat(relocated.get(), equalTo(true)); - assertFalse(shard.isPrimaryMode()); + assertTrue(shard.isRelocatedPrimary()); assertThat(shard.getActiveOperationsCount(), equalTo(0)); for (Thread indexThread : indexThreads) { @@ -1577,7 +1577,7 @@ protected void doRun() throws Exception { cyclicBarrier.await(); relocationThread.join(); cancellingThread.join(); - if (shard.isPrimaryMode() == false) { + if (shard.isRelocatedPrimary()) { logger.debug("shard was relocated successfully"); assertThat(cancellingException.get(), instanceOf(IllegalIndexShardStateException.class)); assertThat("current routing:" + shard.routingEntry(), shard.routingEntry().relocating(), equalTo(true)); @@ -1858,7 +1858,7 @@ public void testRecoveryFailsAfterMovingToRelocatedState() throws InterruptedExc ShardRouting inRecoveryRouting = ShardRoutingHelper.relocate(origRouting, "some_node"); IndexShardTestCase.updateRoutingEntry(shard, inRecoveryRouting); shard.relocated(primaryContext -> {}); - assertFalse(shard.isPrimaryMode()); + assertTrue(shard.isRelocatedPrimary()); try { IndexShardTestCase.updateRoutingEntry(shard, origRouting); fail("Expected IndexShardRelocatedException"); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 15c9109f88466..f0644b029c3dc 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -395,7 +395,7 @@ public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Started() throws IOE final IndexShard shard = mock(IndexShard.class); when(shard.seqNoStats()).thenReturn(mock(SeqNoStats.class)); when(shard.segmentStats(anyBoolean())).thenReturn(mock(SegmentsStats.class)); - when(shard.isPrimaryMode()).thenReturn(false); + when(shard.isRelocatedPrimary()).thenReturn(true); when(shard.acquireSafeIndexCommit()).thenReturn(mock(Engine.IndexCommitRef.class)); doAnswer(invocation -> { ((ActionListener)invocation.getArguments()[0]).onResponse(() -> {}); @@ -444,7 +444,7 @@ public void testCancellationsDoesNotLeakPrimaryPermits() throws Exception { final CancellableThreads cancellableThreads = new CancellableThreads(); final IndexShard shard = mock(IndexShard.class); final AtomicBoolean freed = new AtomicBoolean(true); - when(shard.isPrimaryMode()).thenReturn(true); + when(shard.isRelocatedPrimary()).thenReturn(false); doAnswer(invocation -> { freed.set(false); ((ActionListener)invocation.getArguments()[0]).onResponse(() -> freed.set(true)); From 4b82ca7f770da1f44d9b175fb770da58057816c4 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 2 Aug 2018 15:08:11 +0200 Subject: [PATCH 16/17] refer to javadocs of getPendingPrimaryterm --- .../src/main/java/org/elasticsearch/index/shard/IndexShard.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index e78c5c9eb08d1..41d5c51d76ae1 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -193,7 +193,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl protected volatile ShardRouting shardRouting; protected volatile IndexShardState state; - protected volatile long pendingPrimaryTerm; + protected volatile long pendingPrimaryTerm; // see JavaDocs for getPendingPrimaryTerm protected volatile long operationPrimaryTerm; protected final AtomicReference currentEngineReference = new AtomicReference<>(); final EngineFactory engineFactory; From b65ce61f582a8a59a5f62f4978235c88838f461f Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Thu, 2 Aug 2018 15:24:25 +0300 Subject: [PATCH 17/17] fold double execution protection into bumpPrimaryTerm --- .../elasticsearch/index/shard/IndexShard.java | 27 +++++++------------ .../shard/IndexShardOperationPermits.java | 7 ++++- 2 files changed, 15 insertions(+), 19 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 41d5c51d76ae1..f29f17a46dcf4 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -481,14 +481,9 @@ public void updateShardState(final ShardRouting newRouting, assert pendingPrimaryTerm == newPrimaryTerm : "shard term changed on primary. expected [" + newPrimaryTerm + "] but was [" + pendingPrimaryTerm + "]" + ", current routing: " + currentRouting + ", new routing: " + newRouting; - assert operationPrimaryTerm < newPrimaryTerm; + assert operationPrimaryTerm == newPrimaryTerm; try { - synchronized (mutex) { - assert shardRouting.primary(); - // do these updates under the mutex as this otherwise races with subsequent calls of updateShardState - operationPrimaryTerm = newPrimaryTerm; - replicationTracker.activatePrimaryMode(getLocalCheckpoint()); - } + replicationTracker.activatePrimaryMode(getLocalCheckpoint()); /* * If this shard was serving as a replica shard when another shard was promoted to primary then the state of * its local checkpoint tracker was reset during the primary term transition. In particular, the local @@ -540,7 +535,7 @@ public void onFailure(Exception e) { assert this.shardRouting.primary() == false || this.shardRouting.started() == false || // note that we use started and not active to avoid relocating shards - operationPrimaryTerm != pendingPrimaryTerm || + this.indexShardOperationPermits.isBlocked() || // if permits are blocked, we are still transitioning this.replicationTracker.isPrimaryMode() : "a started primary with non-pending operation term must be in primary mode " + this.shardRouting; shardStateUpdated.countDown(); @@ -2227,8 +2222,12 @@ private void bumpPrimaryTerm(long newPrimaryTerm, final Ch indexShardOperationPermits.asyncBlockOperations(30, TimeUnit.MINUTES, () -> { assert operationPrimaryTerm <= pendingPrimaryTerm; termUpdated.await(); - onBlocked.run(); - assert operationPrimaryTerm <= pendingPrimaryTerm; + // indexShardOperationPermits doesn't guarantee that async submissions are executed + // in the order submitted. We need to guard against another term bump + if (operationPrimaryTerm < newPrimaryTerm) { + operationPrimaryTerm = newPrimaryTerm; + onBlocked.run(); + } }, e -> failShard("exception during primary term transition", e)); pendingPrimaryTerm = newPrimaryTerm; @@ -2270,11 +2269,6 @@ public void acquireReplicaOperationPermit(final long opPrimaryTerm, final long g if (opPrimaryTerm > pendingPrimaryTerm) { bumpPrimaryTerm(opPrimaryTerm, () -> { - // a primary promotion, or another primary term transition, might have been triggered concurrently to this - // recheck under the operation permit if we can skip doing this work - if (opPrimaryTerm == pendingPrimaryTerm) { - assert operationPrimaryTerm < pendingPrimaryTerm; - operationPrimaryTerm = pendingPrimaryTerm; updateGlobalCheckpointOnReplica(globalCheckpoint, "primary term transition"); final long currentGlobalCheckpoint = getGlobalCheckpoint(); final long localCheckpoint; @@ -2290,9 +2284,6 @@ public void acquireReplicaOperationPermit(final long opPrimaryTerm, final long g localCheckpoint); getEngine().resetLocalCheckpoint(localCheckpoint); getEngine().rollTranslogGeneration(); - } else { - logger.trace("a primary promotion or concurrent primary term transition has made this reset obsolete"); - } }); } } diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShardOperationPermits.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShardOperationPermits.java index 09c7a04762081..d4c3833b13a58 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShardOperationPermits.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShardOperationPermits.java @@ -19,7 +19,6 @@ package org.elasticsearch.index.shard; -import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.Assertions; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; @@ -29,6 +28,7 @@ import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ThreadContext.StoredContext; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.threadpool.ThreadPool; import java.io.Closeable; @@ -309,6 +309,11 @@ int getActiveOperationsCount() { } } + + synchronized boolean isBlocked() { + return queuedBlockOperations > 0; + } + /** * @return a list of describing each permit that wasn't released yet. The description consist of the debugInfo supplied * when the permit was acquired plus a stack traces that was captured when the permit was request.