Skip to content

Commit 4300539

Browse files
committed
Die with dignity on the Lucene layer
When a fatal error tragically closes an index writer, such an error never makes its way to the uncaught exception handler. This prevents the node from being torn down if an out of memory error or other fatal error is thrown in the Lucene layer. This commit ensures that such events bubble their way up to the uncaught exception handler. Relates #21721
1 parent 5c0201b commit 4300539

File tree

2 files changed

+54
-5
lines changed

2 files changed

+54
-5
lines changed

core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java

Lines changed: 10 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -960,15 +960,21 @@ public IndexCommit acquireIndexCommit(final boolean flushFirst) throws EngineExc
960960
}
961961
}
962962

963+
@SuppressWarnings("finally")
963964
private void failOnTragicEvent(AlreadyClosedException ex) {
964965
// if we are already closed due to some tragic exception
965966
// we need to fail the engine. it might have already been failed before
966967
// but we are double-checking it's failed and closed
967968
if (indexWriter.isOpen() == false && indexWriter.getTragicException() != null) {
968-
final Exception tragedy = indexWriter.getTragicException() instanceof Exception ?
969-
(Exception) indexWriter.getTragicException() :
970-
new Exception(indexWriter.getTragicException());
971-
failEngine("already closed by tragic event on the index writer", tragedy);
969+
if (indexWriter.getTragicException() instanceof Error) {
970+
try {
971+
logger.error("tragic event in index writer", ex);
972+
} finally {
973+
throw (Error) indexWriter.getTragicException();
974+
}
975+
} else {
976+
failEngine("already closed by tragic event on the index writer", (Exception) indexWriter.getTragicException());
977+
}
972978
} else if (translog.isOpen() == false && translog.getTragicException() != null) {
973979
failEngine("already closed by tragic event on the translog", translog.getTragicException());
974980
} else if (failedEngine.get() == null) { // we are closed but the engine is not failed yet?

core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java

Lines changed: 44 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,8 @@
2626
import org.apache.logging.log4j.core.LogEvent;
2727
import org.apache.logging.log4j.core.appender.AbstractAppender;
2828
import org.apache.logging.log4j.core.filter.RegexFilter;
29+
import org.apache.lucene.analysis.Analyzer;
30+
import org.apache.lucene.analysis.Tokenizer;
2931
import org.apache.lucene.analysis.standard.StandardAnalyzer;
3032
import org.apache.lucene.codecs.Codec;
3133
import org.apache.lucene.document.Field;
@@ -112,6 +114,7 @@
112114
import org.junit.After;
113115
import org.junit.Before;
114116

117+
import java.io.IOError;
115118
import java.io.IOException;
116119
import java.io.InputStream;
117120
import java.nio.charset.Charset;
@@ -208,8 +211,12 @@ public void setUp() throws Exception {
208211
}
209212

210213
public EngineConfig copy(EngineConfig config, EngineConfig.OpenMode openMode) {
214+
return copy(config, openMode, config.getAnalyzer());
215+
}
216+
217+
public EngineConfig copy(EngineConfig config, EngineConfig.OpenMode openMode, Analyzer analyzer) {
211218
return new EngineConfig(openMode, config.getShardId(), config.getThreadPool(), config.getIndexSettings(), config.getWarmer(),
212-
config.getStore(), config.getDeletionPolicy(), config.getMergePolicy(), config.getAnalyzer(), config.getSimilarity(),
219+
config.getStore(), config.getDeletionPolicy(), config.getMergePolicy(), analyzer, config.getSimilarity(),
213220
new CodecService(null, logger), config.getEventListener(), config.getTranslogRecoveryPerformer(), config.getQueryCache(),
214221
config.getQueryCachingPolicy(), config.getTranslogConfig(), config.getFlushMergesAfter(), config.getRefreshListeners(),
215222
config.getMaxUnsafeAutoIdTimestamp());
@@ -2521,4 +2528,40 @@ public void afterRefresh(boolean didRefresh) throws IOException {
25212528
assertTrue(internalEngine.failedEngine.get() instanceof MockDirectoryWrapper.FakeIOException);
25222529
}
25232530
}
2531+
2532+
public void testTragicEventErrorBubblesUp() throws IOException {
2533+
engine.close();
2534+
final AtomicBoolean failWithFatalError = new AtomicBoolean(true);
2535+
final VirtualMachineError error = randomFrom(
2536+
new InternalError(),
2537+
new OutOfMemoryError(),
2538+
new StackOverflowError(),
2539+
new UnknownError());
2540+
engine = new InternalEngine(copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, new Analyzer() {
2541+
@Override
2542+
protected TokenStreamComponents createComponents(String fieldName) {
2543+
return new TokenStreamComponents(new Tokenizer() {
2544+
@Override
2545+
public boolean incrementToken() throws IOException {
2546+
if (failWithFatalError.get()) {
2547+
throw error;
2548+
} else {
2549+
throw new AssertionError("should not get to this point");
2550+
}
2551+
}
2552+
});
2553+
}
2554+
}));
2555+
final Document document = testDocument();
2556+
document.add(new TextField("value", "test", Field.Store.YES));
2557+
final ParsedDocument firstDoc = testParsedDocument("1", "1", "test", null, -1, -1, document, B_1, null);
2558+
final Engine.Index first = new Engine.Index(newUid("1"), firstDoc);
2559+
expectThrows(error.getClass(), () -> engine.index(first));
2560+
failWithFatalError.set(false);
2561+
final ParsedDocument secondDoc = testParsedDocument("2", "2", "test", null, -1, -1, document, B_1, null);
2562+
final Engine.Index second = new Engine.Index(newUid("2"), secondDoc);
2563+
expectThrows(error.getClass(), () -> engine.index(second));
2564+
assertNull(engine.failedEngine.get());
2565+
}
2566+
25242567
}

0 commit comments

Comments
 (0)