@@ -213,8 +213,11 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
213
213
214
214
protected volatile ShardRouting shardRouting ;
215
215
protected volatile IndexShardState state ;
216
+ // ensure happens-before relation between addRefreshListener() and postRecovery()
217
+ private final Object postRecoveryMutex = new Object ();
216
218
private volatile long pendingPrimaryTerm ; // see JavaDocs for getPendingPrimaryTerm
217
- protected final AtomicReference <Engine > currentEngineReference = new AtomicReference <>();
219
+ private final Object engineMutex = new Object (); // lock ordering: engineMutex -> mutex
220
+ private final AtomicReference <Engine > currentEngineReference = new AtomicReference <>();
218
221
final EngineFactory engineFactory ;
219
222
220
223
private final IndexingOperationListener indexingOperationListeners ;
@@ -1192,20 +1195,23 @@ public Engine.IndexCommitRef acquireSafeIndexCommit() throws EngineException {
1192
1195
* @throws java.nio.file.NoSuchFileException if one or more files referenced by a commit are not present.
1193
1196
*/
1194
1197
public Store .MetadataSnapshot snapshotStoreMetadata () throws IOException {
1198
+ assert Thread .holdsLock (mutex ) == false : "snapshotting store metadata under mutex" ;
1195
1199
Engine .IndexCommitRef indexCommit = null ;
1196
1200
store .incRef ();
1197
1201
try {
1198
- Engine engine ;
1199
- synchronized (mutex ) {
1202
+ synchronized (engineMutex ) {
1200
1203
// if the engine is not running, we can access the store directly, but we need to make sure no one starts
1201
- // the engine on us. If the engine is running, we can get a snapshot via the deletion policy which is initialized.
1202
- // That can be done out of mutex, since the engine can be closed half way.
1203
- engine = getEngineOrNull ();
1204
- if (engine == null ) {
1204
+ // the engine on us. If the engine is running, we can get a snapshot via the deletion policy of the engine.
1205
+ synchronized (mutex ) {
1206
+ final Engine engine = getEngineOrNull ();
1207
+ if (engine != null ) {
1208
+ indexCommit = engine .acquireLastIndexCommit (false );
1209
+ }
1210
+ }
1211
+ if (indexCommit == null ) {
1205
1212
return store .getMetadata (null , true );
1206
1213
}
1207
1214
}
1208
- indexCommit = engine .acquireLastIndexCommit (false );
1209
1215
return store .getMetadata (indexCommit .getIndexCommit ());
1210
1216
} finally {
1211
1217
store .decRef ();
@@ -1334,23 +1340,24 @@ public void close(String reason, boolean flushEngine) throws IOException {
1334
1340
}
1335
1341
}
1336
1342
1337
- public IndexShard postRecovery (String reason )
1338
- throws IndexShardStartedException , IndexShardRelocatedException , IndexShardClosedException {
1339
- synchronized (mutex ) {
1340
- if (state == IndexShardState .CLOSED ) {
1341
- throw new IndexShardClosedException (shardId );
1342
- }
1343
- if (state == IndexShardState .STARTED ) {
1344
- throw new IndexShardStartedException (shardId );
1345
- }
1343
+ public void postRecovery (String reason ) throws IndexShardStartedException , IndexShardRelocatedException , IndexShardClosedException {
1344
+ synchronized (postRecoveryMutex ) {
1346
1345
// we need to refresh again to expose all operations that were index until now. Otherwise
1347
1346
// we may not expose operations that were indexed with a refresh listener that was immediately
1348
- // responded to in addRefreshListener.
1347
+ // responded to in addRefreshListener. The refresh must happen under the same mutex used in addRefreshListener
1348
+ // and before moving this shard to POST_RECOVERY state (i.e., allow to read from this shard).
1349
1349
getEngine ().refresh ("post_recovery" );
1350
- recoveryState .setStage (RecoveryState .Stage .DONE );
1351
- changeState (IndexShardState .POST_RECOVERY , reason );
1350
+ synchronized (mutex ) {
1351
+ if (state == IndexShardState .CLOSED ) {
1352
+ throw new IndexShardClosedException (shardId );
1353
+ }
1354
+ if (state == IndexShardState .STARTED ) {
1355
+ throw new IndexShardStartedException (shardId );
1356
+ }
1357
+ recoveryState .setStage (RecoveryState .Stage .DONE );
1358
+ changeState (IndexShardState .POST_RECOVERY , reason );
1359
+ }
1352
1360
}
1353
- return this ;
1354
1361
}
1355
1362
1356
1363
/**
@@ -1583,6 +1590,7 @@ public void openEngineAndSkipTranslogRecovery() throws IOException {
1583
1590
}
1584
1591
1585
1592
private void innerOpenEngineAndTranslog (LongSupplier globalCheckpointSupplier ) throws IOException {
1593
+ assert Thread .holdsLock (mutex ) == false : "opening engine under mutex" ;
1586
1594
if (state != IndexShardState .RECOVERING ) {
1587
1595
throw new IndexShardNotRecoveringException (shardId , state );
1588
1596
}
@@ -1595,16 +1603,24 @@ private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier) t
1595
1603
assert recoveryState .getRecoverySource ().expectEmptyRetentionLeases () == false || getRetentionLeases ().leases ().isEmpty ()
1596
1604
: "expected empty set of retention leases with recovery source [" + recoveryState .getRecoverySource ()
1597
1605
+ "] but got " + getRetentionLeases ();
1598
- synchronized (mutex ) {
1599
- verifyNotClosed ();
1600
- assert currentEngineReference .get () == null : "engine is running" ;
1606
+ synchronized (engineMutex ) {
1601
1607
// we must create a new engine under mutex (see IndexShard#snapshotStoreMetadata).
1602
1608
final Engine newEngine = engineFactory .newReadWriteEngine (config );
1603
- onNewEngine (newEngine );
1604
- currentEngineReference .set (newEngine );
1605
- // We set active because we are now writing operations to the engine; this way,
1606
- // if we go idle after some time and become inactive, we still give sync'd flush a chance to run.
1607
- active .set (true );
1609
+ synchronized (mutex ) {
1610
+ try {
1611
+ verifyNotClosed ();
1612
+ assert currentEngineReference .get () == null : "engine is running" ;
1613
+ onNewEngine (newEngine );
1614
+ currentEngineReference .set (newEngine );
1615
+ // We set active because we are now writing operations to the engine; this way,
1616
+ // if we go idle after some time and become inactive, we still give sync'd flush a chance to run.
1617
+ active .set (true );
1618
+ } finally {
1619
+ if (currentEngineReference .get () != newEngine ) {
1620
+ newEngine .close ();
1621
+ }
1622
+ }
1623
+ }
1608
1624
}
1609
1625
// time elapses after the engine is created above (pulling the config settings) until we set the engine reference, during
1610
1626
// which settings changes could possibly have happened, so here we forcefully push any config changes to the new engine.
@@ -1627,6 +1643,7 @@ private boolean assertSequenceNumbersInCommit() throws IOException {
1627
1643
}
1628
1644
1629
1645
private void onNewEngine (Engine newEngine ) {
1646
+ assert Thread .holdsLock (engineMutex );
1630
1647
refreshListeners .setCurrentRefreshLocationSupplier (newEngine ::getTranslogLastWriteLocation );
1631
1648
}
1632
1649
@@ -2675,7 +2692,13 @@ private DocumentMapperForType docMapper(String type) {
2675
2692
}
2676
2693
2677
2694
private EngineConfig newEngineConfig (LongSupplier globalCheckpointSupplier ) {
2678
- Sort indexSort = indexSortSupplier .get ();
2695
+ final Sort indexSort = indexSortSupplier .get ();
2696
+ final Engine .Warmer warmer = reader -> {
2697
+ assert Thread .holdsLock (mutex ) == false : "warming engine under mutex" ;
2698
+ if (this .warmer != null ) {
2699
+ this .warmer .warm (reader );
2700
+ }
2701
+ };
2679
2702
return new EngineConfig (shardId , shardRouting .allocationId ().getId (),
2680
2703
threadPool , indexSettings , warmer , store , indexSettings .getMergePolicy (),
2681
2704
mapperService != null ? mapperService .indexAnalyzer () : null ,
@@ -3237,10 +3260,10 @@ public void addRefreshListener(Translog.Location location, Consumer<Boolean> lis
3237
3260
if (isReadAllowed ()) {
3238
3261
readAllowed = true ;
3239
3262
} else {
3240
- // check again under mutex . this is important to create a happens before relationship
3263
+ // check again under postRecoveryMutex . this is important to create a happens before relationship
3241
3264
// between the switch to POST_RECOVERY + associated refresh. Otherwise we may respond
3242
3265
// to a listener before a refresh actually happened that contained that operation.
3243
- synchronized (mutex ) {
3266
+ synchronized (postRecoveryMutex ) {
3244
3267
readAllowed = isReadAllowed ();
3245
3268
}
3246
3269
}
@@ -3305,6 +3328,7 @@ public ParsedDocument newNoopTombstoneDoc(String reason) {
3305
3328
* Rollback the current engine to the safe commit, then replay local translog up to the global checkpoint.
3306
3329
*/
3307
3330
void resetEngineToGlobalCheckpoint () throws IOException {
3331
+ assert Thread .holdsLock (engineMutex ) == false : "resetting engine under mutex" ;
3308
3332
assert getActiveOperationsCount () == OPERATIONS_BLOCKED
3309
3333
: "resetting engine without blocking operations; active operations are [" + getActiveOperations () + ']' ;
3310
3334
sync (); // persist the global checkpoint to disk
@@ -3316,23 +3340,28 @@ assert getActiveOperationsCount() == OPERATIONS_BLOCKED
3316
3340
SetOnce <Engine > newEngineReference = new SetOnce <>();
3317
3341
final long globalCheckpoint = getLastKnownGlobalCheckpoint ();
3318
3342
assert globalCheckpoint == getLastSyncedGlobalCheckpoint ();
3319
- synchronized (mutex ) {
3320
- verifyNotClosed ();
3321
- // we must create both new read-only engine and new read-write engine under mutex to ensure snapshotStoreMetadata,
3343
+ synchronized (engineMutex ) {
3344
+ // we must create both new read-only engine and new read-write engine under engineMutex to ensure snapshotStoreMetadata,
3322
3345
// acquireXXXCommit and close works.
3323
3346
final Engine readOnlyEngine =
3324
3347
new ReadOnlyEngine (newEngineConfig (replicationTracker ), seqNoStats , translogStats , false , Function .identity ()) {
3325
3348
@ Override
3326
3349
public IndexCommitRef acquireLastIndexCommit (boolean flushFirst ) {
3327
- synchronized (mutex ) {
3350
+ synchronized (engineMutex ) {
3351
+ if (newEngineReference .get () == null ) {
3352
+ throw new AlreadyClosedException ("engine was closed" );
3353
+ }
3328
3354
// ignore flushFirst since we flushed above and we do not want to interfere with ongoing translog replay
3329
3355
return newEngineReference .get ().acquireLastIndexCommit (false );
3330
3356
}
3331
3357
}
3332
3358
3333
3359
@ Override
3334
3360
public IndexCommitRef acquireSafeIndexCommit () {
3335
- synchronized (mutex ) {
3361
+ synchronized (engineMutex ) {
3362
+ if (newEngineReference .get () == null ) {
3363
+ throw new AlreadyClosedException ("engine was closed" );
3364
+ }
3336
3365
return newEngineReference .get ().acquireSafeIndexCommit ();
3337
3366
}
3338
3367
}
@@ -3349,9 +3378,28 @@ public void close() throws IOException {
3349
3378
IOUtils .close (super ::close , newEngine );
3350
3379
}
3351
3380
};
3352
- IOUtils .close (currentEngineReference .getAndSet (readOnlyEngine ));
3353
- newEngineReference .set (engineFactory .newReadWriteEngine (newEngineConfig (replicationTracker )));
3354
- onNewEngine (newEngineReference .get ());
3381
+ synchronized (mutex ) {
3382
+ try {
3383
+ verifyNotClosed ();
3384
+ IOUtils .close (currentEngineReference .getAndSet (readOnlyEngine ));
3385
+ } finally {
3386
+ if (currentEngineReference .get () != readOnlyEngine ) {
3387
+ readOnlyEngine .close ();
3388
+ }
3389
+ }
3390
+ }
3391
+ final Engine newReadWriteEngine = engineFactory .newReadWriteEngine (newEngineConfig (replicationTracker ));
3392
+ synchronized (mutex ) {
3393
+ try {
3394
+ verifyNotClosed ();
3395
+ newEngineReference .set (newReadWriteEngine );
3396
+ onNewEngine (newReadWriteEngine );
3397
+ } finally {
3398
+ if (newEngineReference .get () != newReadWriteEngine ) {
3399
+ newReadWriteEngine .close (); // shard was closed
3400
+ }
3401
+ }
3402
+ }
3355
3403
}
3356
3404
final Engine .TranslogRecoveryRunner translogRunner = (engine , snapshot ) -> runTranslogRecovery (
3357
3405
engine , snapshot , Engine .Operation .Origin .LOCAL_RESET , () -> {
0 commit comments