Skip to content

Commit bd56da8

Browse files
committed
Create and swap engine in two steps
1 parent 16cd233 commit bd56da8

File tree

3 files changed

+62
-247
lines changed

3 files changed

+62
-247
lines changed

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

-84
This file was deleted.

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

+62-47
Original file line numberDiff line numberDiff line change
@@ -214,7 +214,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
214214
protected volatile IndexShardState state;
215215
private volatile long pendingPrimaryTerm; // see JavaDocs for getPendingPrimaryTerm
216216
private final Object engineMutex = new Object();
217-
private final EngineReference currentEngineReference = new EngineReference();
217+
private final AtomicReference<Engine> currentEngineReference = new AtomicReference<>();
218218
final EngineFactory engineFactory;
219219

220220
private final IndexingOperationListener indexingOperationListeners;
@@ -1191,20 +1191,23 @@ public Engine.IndexCommitRef acquireSafeIndexCommit() throws EngineException {
11911191
* @throws java.nio.file.NoSuchFileException if one or more files referenced by a commit are not present.
11921192
*/
11931193
public Store.MetadataSnapshot snapshotStoreMetadata() throws IOException {
1194+
assert Thread.holdsLock(mutex) == false : "snapshotting store metadata under mutex";
11941195
Engine.IndexCommitRef indexCommit = null;
11951196
store.incRef();
11961197
try {
1197-
Engine engine;
11981198
synchronized (engineMutex) {
11991199
// if the engine is not running, we can access the store directly, but we need to make sure no one starts
1200-
// the engine on us. If the engine is running, we can get a snapshot via the deletion policy which is initialized.
1201-
// That can be done out of mutex, since the engine can be closed half way.
1202-
engine = getEngineOrNull();
1203-
if (engine == null) {
1200+
// the engine on us. If the engine is running, we can get a snapshot via the deletion policy of the engine.
1201+
synchronized (mutex) {
1202+
final Engine engine = getEngineOrNull();
1203+
if (engine != null) {
1204+
indexCommit = engine.acquireLastIndexCommit(false);
1205+
}
1206+
}
1207+
if (indexCommit == null) {
12041208
return store.getMetadata(null, true);
12051209
}
12061210
}
1207-
indexCommit = engine.acquireLastIndexCommit(false);
12081211
return store.getMetadata(indexCommit.getIndexCommit());
12091212
} finally {
12101213
store.decRef();
@@ -1318,14 +1321,15 @@ public void close(String reason, boolean flushEngine) throws IOException {
13181321
try {
13191322
changeState(IndexShardState.CLOSED, reason);
13201323
} finally {
1324+
final Engine engine = this.currentEngineReference.getAndSet(null);
13211325
try {
1322-
if (flushEngine) {
1323-
currentEngineReference.flushAndClose();
1326+
if (engine != null && flushEngine) {
1327+
engine.flushAndClose();
13241328
}
13251329
} finally {
13261330
// playing safe here and close the engine even if the above succeeds - close can be called multiple times
13271331
// Also closing refreshListeners to prevent us from accumulating any more listeners
1328-
IOUtils.close(currentEngineReference, globalCheckpointListeners, refreshListeners);
1332+
IOUtils.close(engine, globalCheckpointListeners, refreshListeners);
13291333
indexShardOperationPermits.close();
13301334
}
13311335
}
@@ -1347,7 +1351,7 @@ public IndexShard postRecovery(String reason)
13471351
// we need to refresh again to expose all operations that were index until now. Otherwise
13481352
// we may not expose operations that were indexed with a refresh listener that was immediately
13491353
// responded to in addRefreshListener.
1350-
refresh("post_recovery");
1354+
getEngine().refresh("post_recovery");
13511355
return this;
13521356
}
13531357

@@ -1420,7 +1424,9 @@ public long recoverLocallyUpToGlobalCheckpoint() {
14201424
getEngine().recoverFromTranslog(translogRecoveryRunner, globalCheckpoint);
14211425
logger.trace("shard locally recovered up to {}", getEngine().getSeqNoStats(globalCheckpoint));
14221426
} finally {
1423-
currentEngineReference.swapReference(null);
1427+
synchronized (mutex) {
1428+
IOUtils.close(currentEngineReference.getAndSet(null));
1429+
}
14241430
}
14251431
} catch (Exception e) {
14261432
logger.debug(new ParameterizedMessage("failed to recover shard locally up to global checkpoint {}", globalCheckpoint), e);
@@ -1579,7 +1585,7 @@ public void openEngineAndSkipTranslogRecovery() throws IOException {
15791585
}
15801586

15811587
private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier) throws IOException {
1582-
assert Thread.holdsLock(mutex) == false : "opening engine under mutex [" + Thread.currentThread() + "]";
1588+
assert Thread.holdsLock(mutex) == false : "opening engine under mutex";
15831589
if (state != IndexShardState.RECOVERING) {
15841590
throw new IndexShardNotRecoveringException(shardId, state);
15851591
}
@@ -1593,19 +1599,25 @@ private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier) t
15931599
: "expected empty set of retention leases with recovery source [" + recoveryState.getRecoverySource()
15941600
+ "] but got " + getRetentionLeases();
15951601
synchronized (engineMutex) {
1596-
assert currentEngineReference.get() == null : "engine is running";
15971602
// we must create a new engine under mutex (see IndexShard#snapshotStoreMetadata).
1598-
Engine newEngine = engineFactory.newReadWriteEngine(config);
1599-
onNewEngine(newEngine);
1603+
final Engine newEngine = engineFactory.newReadWriteEngine(config);
1604+
boolean success = false;
16001605
try {
1601-
currentEngineReference.swapReference(newEngine);
1602-
newEngine = null;
1606+
synchronized (mutex) {
1607+
verifyNotClosed();
1608+
assert currentEngineReference.get() == null : "engine is running";
1609+
onNewEngine(newEngine);
1610+
currentEngineReference.set(newEngine);
1611+
// We set active because we are now writing operations to the engine; this way,
1612+
// if we go idle after some time and become inactive, we still give sync'd flush a chance to run.
1613+
active.set(true);
1614+
success = true;
1615+
}
16031616
} finally {
1604-
IOUtils.close(newEngine);
1617+
if (success == false) {
1618+
newEngine.close();
1619+
}
16051620
}
1606-
// We set active because we are now writing operations to the engine; this way,
1607-
// if we go idle after some time and become inactive, we still give sync'd flush a chance to run.
1608-
active.set(true);
16091621
}
16101622
// time elapses after the engine is created above (pulling the config settings) until we set the engine reference, during
16111623
// which settings changes could possibly have happened, so here we forcefully push any config changes to the new engine.
@@ -1628,6 +1640,7 @@ private boolean assertSequenceNumbersInCommit() throws IOException {
16281640
}
16291641

16301642
private void onNewEngine(Engine newEngine) {
1643+
assert Thread.holdsLock(engineMutex);
16311644
refreshListeners.setCurrentRefreshLocationSupplier(newEngine::getTranslogLastWriteLocation);
16321645
}
16331646

@@ -1637,7 +1650,7 @@ private void onNewEngine(Engine newEngine) {
16371650
public void performRecoveryRestart() throws IOException {
16381651
synchronized (mutex) {
16391652
assert refreshListeners.pendingCount() == 0 : "we can't restart with pending listeners";
1640-
currentEngineReference.swapReference(null);
1653+
IOUtils.close(currentEngineReference.getAndSet(null));
16411654
resetRecoveryStage();
16421655
}
16431656
}
@@ -2671,8 +2684,10 @@ private DocumentMapperForType docMapper(String type) {
26712684
private EngineConfig newEngineConfig(LongSupplier globalCheckpointSupplier) {
26722685
final Sort indexSort = indexSortSupplier.get();
26732686
final Engine.Warmer warmer = reader -> {
2674-
assert Thread.holdsLock(mutex) == false : "warming engine under mutex [" + Thread.currentThread() + "]";
2675-
this.warmer.warm(reader);
2687+
assert Thread.holdsLock(mutex) == false : "warming engine under mutex";
2688+
if (this.warmer != null) {
2689+
this.warmer.warm(reader);
2690+
}
26762691
};
26772692
return new EngineConfig(shardId, shardRouting.allocationId().getId(),
26782693
threadPool, indexSettings, warmer, store, indexSettings.getMergePolicy(),
@@ -3303,7 +3318,7 @@ public ParsedDocument newNoopTombstoneDoc(String reason) {
33033318
* Rollback the current engine to the safe commit, then replay local translog up to the global checkpoint.
33043319
*/
33053320
void resetEngineToGlobalCheckpoint() throws IOException {
3306-
assert Thread.holdsLock(mutex) == false : "resetting engine under mutex [" + Thread.currentThread() + "]";
3321+
assert Thread.holdsLock(engineMutex) == false : "resetting engine under mutex";
33073322
assert getActiveOperationsCount() == OPERATIONS_BLOCKED
33083323
: "resetting engine without blocking operations; active operations are [" + getActiveOperations() + ']';
33093324
sync(); // persist the global checkpoint to disk
@@ -3318,41 +3333,46 @@ assert getActiveOperationsCount() == OPERATIONS_BLOCKED
33183333
synchronized (engineMutex) {
33193334
// we must create both new read-only engine and new read-write engine under engineMutex to ensure snapshotStoreMetadata,
33203335
// acquireXXXCommit and close works.
3321-
Engine readOnlyEngine =
3336+
final Engine readOnlyEngine =
33223337
new ReadOnlyEngine(newEngineConfig(replicationTracker), seqNoStats, translogStats, false, Function.identity()) {
33233338
@Override
33243339
public IndexCommitRef acquireLastIndexCommit(boolean flushFirst) {
3325-
synchronized (engineMutex) {
3340+
synchronized (mutex) {
33263341
// ignore flushFirst since we flushed above and we do not want to interfere with ongoing translog replay
33273342
return newEngineReference.get().acquireLastIndexCommit(false);
33283343
}
33293344
}
33303345

33313346
@Override
33323347
public IndexCommitRef acquireSafeIndexCommit() {
3333-
synchronized (engineMutex) {
3348+
synchronized (mutex) {
33343349
return newEngineReference.get().acquireSafeIndexCommit();
33353350
}
33363351
}
33373352

33383353
@Override
33393354
public void close() throws IOException {
3340-
Engine newEngine;
3341-
synchronized (engineMutex) {
3342-
newEngine = newEngineReference.get();
3343-
if (newEngine == currentEngineReference.get()) {
3344-
// we successfully installed the new engine so do not close it.
3345-
newEngine = null;
3346-
}
3355+
assert Thread.holdsLock(mutex);
3356+
3357+
Engine newEngine = newEngineReference.get();
3358+
if (newEngine == currentEngineReference.get()) {
3359+
// we successfully installed the new engine so do not close it.
3360+
newEngine = null;
33473361
}
33483362
IOUtils.close(super::close, newEngine);
33493363
}
33503364
};
3365+
boolean success = false;
33513366
try {
3352-
currentEngineReference.swapReference(readOnlyEngine);
3353-
readOnlyEngine = null;
3367+
synchronized (mutex) {
3368+
verifyNotClosed();
3369+
IOUtils.close(currentEngineReference.getAndSet(readOnlyEngine));
3370+
success = true;
3371+
}
33543372
} finally {
3355-
IOUtils.close(readOnlyEngine);
3373+
if (success == false) {
3374+
readOnlyEngine.close();
3375+
}
33563376
}
33573377
newEngineReference.set(engineFactory.newReadWriteEngine(newEngineConfig(replicationTracker)));
33583378
onNewEngine(newEngineReference.get());
@@ -3362,14 +3382,9 @@ public void close() throws IOException {
33623382
// TODO: add a dedicate recovery stats for the reset translog
33633383
});
33643384
newEngineReference.get().recoverFromTranslog(translogRunner, globalCheckpoint);
3365-
synchronized (engineMutex) {
3366-
Engine newEngine = newEngineReference.get();
3367-
try {
3368-
currentEngineReference.swapReference(newEngine);
3369-
newEngine = null;
3370-
} finally {
3371-
IOUtils.close(newEngine);
3372-
}
3385+
synchronized (mutex) {
3386+
verifyNotClosed();
3387+
IOUtils.close(currentEngineReference.getAndSet(newEngineReference.get()));
33733388
// We set active because we are now writing operations to the engine; this way,
33743389
// if we go idle after some time and become inactive, we still give sync'd flush a chance to run.
33753390
active.set(true);

0 commit comments

Comments
 (0)