Skip to content

Commit 8d41d4d

Browse files
committed
A replica can be promoted and started in one cluster state update (#32042)
When a replica is fully recovered (i.e., in `POST_RECOVERY` state) we send a request to the master to start the shard. The master changes the state of the replica and publishes a cluster state to that effect. In certain cases, that cluster state can be processed on the node hosting the replica *together* with a cluster state that promotes that, now started, replica to a primary. This can happen due to cluster state batched processing or if the master died after having committed the cluster state that starts the shard but before publishing it to the node with the replica. If the master also held the primary shard, the new master node will remove the primary (as it failed) and will also immediately promote the replica (thinking it is started). Sadly our code in IndexShard didn't allow for this which caused [assertions](https://github.com/elastic/elasticsearch/blob/13917162ad5c59a96ccb4d6a81a5044546c45c22/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java#L482) to be tripped in some of our tests runs.
1 parent e598cef commit 8d41d4d

File tree

8 files changed

+137
-87
lines changed

8 files changed

+137
-87
lines changed

server/src/main/java/org/elasticsearch/index/shard/IndexShard.java

+25-17
Original file line numberDiff line numberDiff line change
@@ -47,7 +47,6 @@
4747
import org.elasticsearch.action.admin.indices.upgrade.post.UpgradeRequest;
4848
import org.elasticsearch.cluster.metadata.IndexMetaData;
4949
import org.elasticsearch.cluster.metadata.MappingMetaData;
50-
import org.elasticsearch.cluster.node.DiscoveryNode;
5150
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
5251
import org.elasticsearch.cluster.routing.RecoverySource;
5352
import org.elasticsearch.cluster.routing.RecoverySource.SnapshotRecoverySource;
@@ -403,21 +402,10 @@ public void updateShardState(final ShardRouting newRouting,
403402
if (state == IndexShardState.POST_RECOVERY && newRouting.active()) {
404403
assert currentRouting.active() == false : "we are in POST_RECOVERY, but our shard routing is active " + currentRouting;
405404

406-
if (newRouting.primary()) {
407-
final DiscoveryNode recoverySourceNode = recoveryState.getSourceNode();
408-
final Engine engine = getEngine();
409-
if (currentRouting.isRelocationTarget() == false || recoverySourceNode.getVersion().before(Version.V_6_0_0_alpha1)) {
410-
// there was no primary context hand-off in < 6.0.0, need to manually activate the shard
411-
replicationTracker.activatePrimaryMode(getEngine().getLocalCheckpointTracker().getCheckpoint());
412-
}
413-
if (currentRouting.isRelocationTarget() == true && recoverySourceNode.getVersion().before(Version.V_6_0_0_alpha1)) {
414-
// Flush the translog as it may contain operations with no sequence numbers. We want to make sure those
415-
// operations will never be replayed as part of peer recovery to avoid an arbitrary mixture of operations with seq#
416-
// (due to active indexing) and operations without a seq# coming from the translog. We therefore flush
417-
// to create a lucene commit point to an empty translog file.
418-
engine.flush(false, true);
419-
}
420-
}
405+
assert currentRouting.isRelocationTarget() == false || currentRouting.primary() == false ||
406+
recoveryState.getSourceNode().getVersion().before(Version.V_6_0_0_alpha1) ||
407+
replicationTracker.isPrimaryMode() :
408+
"a primary relocation is completed by the master, but primary mode is not active " + currentRouting;
421409

422410
changeState(IndexShardState.STARTED, "global state is [" + newRouting.state() + "]");
423411
} else if (currentRouting.primary() && currentRouting.relocating() && replicationTracker.isPrimaryMode() == false &&
@@ -433,7 +421,22 @@ public void updateShardState(final ShardRouting newRouting,
433421
final CountDownLatch shardStateUpdated = new CountDownLatch(1);
434422

435423
if (newRouting.primary()) {
436-
if (newPrimaryTerm != primaryTerm) {
424+
if (newPrimaryTerm == primaryTerm) {
425+
if (currentRouting.initializing() && newRouting.active()) {
426+
if (currentRouting.isRelocationTarget() == false) {
427+
// the master started a recovering primary, activate primary mode.
428+
replicationTracker.activatePrimaryMode(getLocalCheckpoint());
429+
} else if (recoveryState.getSourceNode().getVersion().before(Version.V_6_0_0_alpha1)) {
430+
// there was no primary context hand-off in < 6.0.0, need to manually activate the shard
431+
replicationTracker.activatePrimaryMode(getLocalCheckpoint());
432+
// Flush the translog as it may contain operations with no sequence numbers. We want to make sure those
433+
// operations will never be replayed as part of peer recovery to avoid an arbitrary mixture of operations with
434+
// seq# (due to active indexing) and operations without a seq# coming from the translog. We therefore flush
435+
// to create a lucene commit point to an empty translog file.
436+
getEngine().flush(false, true);
437+
}
438+
}
439+
} else {
437440
assert currentRouting.primary() == false : "term is only increased as part of primary promotion";
438441
/* Note that due to cluster state batching an initializing primary shard term can failed and re-assigned
439442
* in one state causing it's term to be incremented. Note that if both current shard state and new
@@ -531,6 +534,11 @@ public void onFailure(Exception e) {
531534
}
532535
// set this last, once we finished updating all internal state.
533536
this.shardRouting = newRouting;
537+
538+
assert this.shardRouting.primary() == false ||
539+
this.shardRouting.started() == false || // note that we use started and not active to avoid relocating shards
540+
this.replicationTracker.isPrimaryMode()
541+
: "an started primary must be in primary mode " + this.shardRouting;
534542
shardStateUpdated.countDown();
535543
}
536544
if (currentRouting != null && currentRouting.active() == false && newRouting.active()) {

server/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java

+6-4
Original file line numberDiff line numberDiff line change
@@ -73,7 +73,6 @@
7373
import org.elasticsearch.threadpool.ThreadPool;
7474

7575
import java.io.IOException;
76-
import java.util.ArrayList;
7776
import java.util.Collections;
7877
import java.util.HashSet;
7978
import java.util.Iterator;
@@ -265,7 +264,7 @@ public synchronized IndexShard addReplicaWithExistingPath(final ShardPath shardP
265264
RecoverySource.PeerRecoverySource.INSTANCE);
266265

267266
final IndexShard newReplica =
268-
newShard(shardRouting, shardPath, indexMetaData, null, getEngineFactory(shardRouting), () -> {}, EMPTY_EVENT_LISTENER);
267+
newShard(shardRouting, shardPath, indexMetaData, null, getEngineFactory(shardRouting), () -> {}, EMPTY_EVENT_LISTENER);
269268
replicas.add(newReplica);
270269
updateAllocationIDsOnPrimary();
271270
return newReplica;
@@ -341,8 +340,11 @@ public void recoverReplica(
341340
IndexShard replica,
342341
BiFunction<IndexShard, DiscoveryNode, RecoveryTarget> targetSupplier,
343342
boolean markAsRecovering) throws IOException {
344-
ESIndexLevelReplicationTestCase.this.recoverReplica(replica, primary, targetSupplier, markAsRecovering, activeIds(),
345-
routingTable(Function.identity()));
343+
final IndexShardRoutingTable routingTable = routingTable(Function.identity());
344+
final Set<String> inSyncIds = activeIds();
345+
ESIndexLevelReplicationTestCase.this.recoverUnstartedReplica(replica, primary, targetSupplier, markAsRecovering, inSyncIds,
346+
routingTable);
347+
ESIndexLevelReplicationTestCase.this.startReplicaAfterRecovery(replica, primary, inSyncIds, routingTable);
346348
}
347349

348350
public synchronized DiscoveryNode getPrimaryNode() {

server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -361,7 +361,7 @@ public void testSeqNoCollision() throws Exception {
361361
logger.info("--> Promote replica2 as the primary");
362362
shards.promoteReplicaToPrimary(replica2);
363363
logger.info("--> Recover replica3 from replica2");
364-
recoverReplica(replica3, replica2);
364+
recoverReplica(replica3, replica2, true);
365365
try (Translog.Snapshot snapshot = getTranslog(replica3).newSnapshot()) {
366366
assertThat(snapshot.totalOperations(), equalTo(initDocs + 1));
367367
assertThat(snapshot.next(), equalTo(op2));

server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java

+22-45
Original file line numberDiff line numberDiff line change
@@ -221,6 +221,7 @@ public void testPersistenceStateMetadataPersistence() throws Exception {
221221
}
222222

223223
public void testFailShard() throws Exception {
224+
allowShardFailures();
224225
IndexShard shard = newStartedShard();
225226
final ShardPath shardPath = shard.shardPath();
226227
assertNotNull(shardPath);
@@ -304,7 +305,8 @@ public void testRejectOperationPermitWithHigherTermWhenNotStarted() throws IOExc
304305
}
305306

306307
public void testPrimaryPromotionDelaysOperations() throws IOException, BrokenBarrierException, InterruptedException {
307-
final IndexShard indexShard = newStartedShard(false);
308+
final IndexShard indexShard = newShard(false);
309+
recoveryEmptyReplica(indexShard, randomBoolean());
308310

309311
final int operations = scaledRandomIntBetween(1, 64);
310312
final CyclicBarrier barrier = new CyclicBarrier(1 + operations);
@@ -348,20 +350,10 @@ public void onFailure(Exception e) {
348350
barrier.await();
349351
latch.await();
350352

351-
// promote the replica
352353
final ShardRouting replicaRouting = indexShard.routingEntry();
353-
final ShardRouting primaryRouting =
354-
newShardRouting(
355-
replicaRouting.shardId(),
356-
replicaRouting.currentNodeId(),
357-
null,
358-
true,
359-
ShardRoutingState.STARTED,
360-
replicaRouting.allocationId());
361-
indexShard.updateShardState(primaryRouting, indexShard.getPrimaryTerm() + 1, (shard, listener) -> {},
362-
0L, Collections.singleton(primaryRouting.allocationId().getId()),
363-
new IndexShardRoutingTable.Builder(primaryRouting.shardId()).addShard(primaryRouting).build(),
364-
Collections.emptySet());
354+
promoteReplica(indexShard, Collections.singleton(replicaRouting.allocationId().getId()),
355+
new IndexShardRoutingTable.Builder(replicaRouting.shardId()).addShard(replicaRouting).build());
356+
365357

366358
final int delayedOperations = scaledRandomIntBetween(1, 64);
367359
final CyclicBarrier delayedOperationsBarrier = new CyclicBarrier(1 + delayedOperations);
@@ -423,8 +415,9 @@ public void onFailure(Exception e) {
423415
* 1) Internal state (ala ReplicationTracker) have been updated
424416
* 2) Primary term is set to the new term
425417
*/
426-
public void testPublishingOrderOnPromotion() throws IOException, BrokenBarrierException, InterruptedException {
427-
final IndexShard indexShard = newStartedShard(false);
418+
public void testPublishingOrderOnPromotion() throws IOException, InterruptedException, BrokenBarrierException {
419+
final IndexShard indexShard = newShard(false);
420+
recoveryEmptyReplica(indexShard, randomBoolean());
428421
final long promotedTerm = indexShard.getPrimaryTerm() + 1;
429422
final CyclicBarrier barrier = new CyclicBarrier(2);
430423
final AtomicBoolean stop = new AtomicBoolean();
@@ -443,18 +436,10 @@ public void testPublishingOrderOnPromotion() throws IOException, BrokenBarrierEx
443436
});
444437
thread.start();
445438

446-
final ShardRouting replicaRouting = indexShard.routingEntry();
447-
final ShardRouting primaryRouting = newShardRouting(replicaRouting.shardId(), replicaRouting.currentNodeId(), null, true,
448-
ShardRoutingState.STARTED, replicaRouting.allocationId());
449-
450-
451-
final Set<String> inSyncAllocationIds = Collections.singleton(primaryRouting.allocationId().getId());
452-
final IndexShardRoutingTable routingTable =
453-
new IndexShardRoutingTable.Builder(primaryRouting.shardId()).addShard(primaryRouting).build();
454439
barrier.await();
455-
// promote the replica
456-
indexShard.updateShardState(primaryRouting, promotedTerm, (shard, listener) -> {}, 0L, inSyncAllocationIds, routingTable,
457-
Collections.emptySet());
440+
final ShardRouting replicaRouting = indexShard.routingEntry();
441+
promoteReplica(indexShard, Collections.singleton(replicaRouting.allocationId().getId()),
442+
new IndexShardRoutingTable.Builder(replicaRouting.shardId()).addShard(replicaRouting).build());
458443

459444
stop.set(true);
460445
thread.join();
@@ -463,7 +448,8 @@ public void testPublishingOrderOnPromotion() throws IOException, BrokenBarrierEx
463448

464449

465450
public void testPrimaryFillsSeqNoGapsOnPromotion() throws Exception {
466-
final IndexShard indexShard = newStartedShard(false);
451+
final IndexShard indexShard = newShard(false);
452+
recoveryEmptyReplica(indexShard, randomBoolean());
467453

468454
// most of the time this is large enough that most of the time there will be at least one gap
469455
final int operations = 1024 - scaledRandomIntBetween(0, 1024);
@@ -474,17 +460,8 @@ public void testPrimaryFillsSeqNoGapsOnPromotion() throws Exception {
474460

475461
// promote the replica
476462
final ShardRouting replicaRouting = indexShard.routingEntry();
477-
final ShardRouting primaryRouting =
478-
newShardRouting(
479-
replicaRouting.shardId(),
480-
replicaRouting.currentNodeId(),
481-
null,
482-
true,
483-
ShardRoutingState.STARTED,
484-
replicaRouting.allocationId());
485-
indexShard.updateShardState(primaryRouting, indexShard.getPrimaryTerm() + 1, (shard, listener) -> {},
486-
0L, Collections.singleton(primaryRouting.allocationId().getId()),
487-
new IndexShardRoutingTable.Builder(primaryRouting.shardId()).addShard(primaryRouting).build(), Collections.emptySet());
463+
promoteReplica(indexShard, Collections.singleton(replicaRouting.allocationId().getId()),
464+
new IndexShardRoutingTable.Builder(replicaRouting.shardId()).addShard(replicaRouting).build());
488465

489466
/*
490467
* This operation completing means that the delay operation executed as part of increasing the primary term has completed and the
@@ -501,7 +478,7 @@ public void onResponse(Releasable releasable) {
501478

502479
@Override
503480
public void onFailure(Exception e) {
504-
throw new RuntimeException(e);
481+
throw new AssertionError(e);
505482
}
506483
},
507484
ThreadPool.Names.GENERIC, "");
@@ -840,7 +817,7 @@ public void testGlobalCheckpointSync() throws IOException {
840817
// add a replica
841818
recoverShardFromStore(primaryShard);
842819
final IndexShard replicaShard = newShard(shardId, false);
843-
recoverReplica(replicaShard, primaryShard);
820+
recoverReplica(replicaShard, primaryShard, true);
844821
final int maxSeqNo = randomIntBetween(0, 128);
845822
for (int i = 0; i <= maxSeqNo; i++) {
846823
primaryShard.getEngine().getLocalCheckpointTracker().generateSeqNo();
@@ -1619,7 +1596,7 @@ public void testPrimaryHandOffUpdatesLocalCheckpoint() throws IOException {
16191596
IndexShardTestCase.updateRoutingEntry(primarySource, primarySource.routingEntry().relocate(randomAlphaOfLength(10), -1));
16201597
final IndexShard primaryTarget = newShard(primarySource.routingEntry().getTargetRelocatingShard());
16211598
updateMappings(primaryTarget, primarySource.indexSettings().getIndexMetaData());
1622-
recoverReplica(primaryTarget, primarySource);
1599+
recoverReplica(primaryTarget, primarySource, true);
16231600

16241601
// check that local checkpoint of new primary is properly tracked after primary relocation
16251602
assertThat(primaryTarget.getLocalCheckpoint(), equalTo(totalOps - 1L));
@@ -2055,7 +2032,7 @@ public long indexTranslogOperations(List<Translog.Operation> operations, int tot
20552032
assertFalse(replica.isSyncNeeded());
20562033
return localCheckpoint;
20572034
}
2058-
}, true);
2035+
}, true, true);
20592036

20602037
closeShards(primary, replica);
20612038
}
@@ -2162,7 +2139,7 @@ public long indexTranslogOperations(List<Translog.Operation> operations, int tot
21622139
assertTrue(replica.isActive());
21632140
return localCheckpoint;
21642141
}
2165-
}, false);
2142+
}, false, true);
21662143

21672144
closeShards(primary, replica);
21682145
}
@@ -2214,7 +2191,7 @@ public void finalizeRecovery(long globalCheckpoint) throws IOException {
22142191
super.finalizeRecovery(globalCheckpoint);
22152192
assertListenerCalled.accept(replica);
22162193
}
2217-
}, false);
2194+
}, false, true);
22182195

22192196
closeShards(primary, replica);
22202197
}

server/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java

+8
Original file line numberDiff line numberDiff line change
@@ -357,6 +357,14 @@ public void updateShardState(ShardRouting shardRouting,
357357
assertTrue("and active shard must stay active, current: " + this.shardRouting + ", got: " + shardRouting,
358358
shardRouting.active());
359359
}
360+
if (this.shardRouting.primary()) {
361+
assertTrue("a primary shard can't be demoted", shardRouting.primary());
362+
} else if (shardRouting.primary()) {
363+
// note: it's ok for a replica in post recovery to be started and promoted at once
364+
// this can happen when the primary failed after we sent the start shard message
365+
assertTrue("a replica can only be promoted when active. current: " + this.shardRouting + " new: " + shardRouting,
366+
shardRouting.active());
367+
}
360368
this.shardRouting = shardRouting;
361369
if (shardRouting.primary()) {
362370
term = newPrimaryTerm;

server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,7 @@ public void testGetStartingSeqNo() throws Exception {
4343
try {
4444
// Empty store
4545
{
46-
recoveryEmptyReplica(replica);
46+
recoveryEmptyReplica(replica, true);
4747
final RecoveryTarget recoveryTarget = new RecoveryTarget(replica, null, null, null);
4848
assertThat(PeerRecoveryTargetService.getStartingSeqNo(logger, recoveryTarget), equalTo(0L));
4949
recoveryTarget.decRef();

server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -261,7 +261,7 @@ public void testPeerRecoverySendSafeCommitInFileBased() throws Exception {
261261
}
262262
IndexShard replicaShard = newShard(primaryShard.shardId(), false);
263263
updateMappings(replicaShard, primaryShard.indexSettings().getIndexMetaData());
264-
recoverReplica(replicaShard, primaryShard);
264+
recoverReplica(replicaShard, primaryShard, true);
265265
List<IndexCommit> commits = DirectoryReader.listCommits(replicaShard.store().directory());
266266
long maxSeqNo = Long.parseLong(commits.get(0).getUserData().get(SequenceNumbers.MAX_SEQ_NO));
267267
assertThat(maxSeqNo, lessThanOrEqualTo(globalCheckpoint));

0 commit comments

Comments
 (0)