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 73e87fe39a1da..f60f181de9007 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -1156,7 +1156,7 @@ public void failEngine(String reason, @Nullable Exception failure) { if (failure != null) { maybeDie(reason, failure); } - if (failEngineLock.tryLock() && store.tryIncRef()) { + if (failEngineLock.tryLock()) { try { if (failedEngine.get() != null) { logger.warn(() -> @@ -1178,11 +1178,19 @@ public void failEngine(String reason, @Nullable Exception failure) { // on the same node that we don't see the corrupted marker file when // the shard is initializing if (Lucene.isCorruptionException(failure)) { - try { - store.markStoreCorrupted(new IOException("failed engine (reason: [" + reason + "])", - ExceptionsHelper.unwrapCorruption(failure))); - } catch (IOException e) { - logger.warn("Couldn't mark store corrupted", e); + if (store.tryIncRef()) { + try { + store.markStoreCorrupted(new IOException("failed engine (reason: [" + reason + "])", + ExceptionsHelper.unwrapCorruption(failure))); + } catch (IOException e) { + logger.warn("Couldn't mark store corrupted", e); + } finally { + store.decRef(); + } + } else { + logger.warn(() -> + new ParameterizedMessage("tried to mark store as corrupted but store is already closed. [{}]", reason), + failure); } } eventListener.onFailedEngine(reason, failure); @@ -1191,8 +1199,6 @@ public void failEngine(String reason, @Nullable Exception failure) { if (failure != null) inner.addSuppressed(failure); // don't bubble up these exceptions up logger.warn("failEngine threw exception", inner); - } finally { - store.decRef(); } } else { logger.debug(() -> new ParameterizedMessage("tried to fail engine but could not acquire lock - engine should " +