Skip to content

Commit 9a2ec69

Browse files
committed
Die with dignity while merging
If an out of memory error is thrown while merging, today we quietly rewrap it into a merge exception and the out of memory error is lost. Instead, we need to rethrow out of memory errors, and in fact any fatal error here, and let those go uncaught so that the node is torn down. This commit causes this to be the case. Relates #27265
1 parent b15484f commit 9a2ec69

File tree

11 files changed

+624
-383
lines changed

11 files changed

+624
-383
lines changed

core/src/main/java/org/elasticsearch/bootstrap/ElasticsearchUncaughtExceptionHandler.java

+1-6
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,6 @@
2121

2222
import org.apache.logging.log4j.Logger;
2323
import org.apache.logging.log4j.message.ParameterizedMessage;
24-
import org.apache.lucene.index.MergePolicy;
2524
import org.elasticsearch.common.SuppressForbidden;
2625
import org.elasticsearch.common.logging.Loggers;
2726

@@ -68,11 +67,7 @@ public void uncaughtException(Thread t, Throwable e) {
6867

6968
// visible for testing
7069
static boolean isFatalUncaught(Throwable e) {
71-
return isFatalCause(e) || (e instanceof MergePolicy.MergeException && isFatalCause(e.getCause()));
72-
}
73-
74-
private static boolean isFatalCause(Throwable cause) {
75-
return cause instanceof Error;
70+
return e instanceof Error;
7671
}
7772

7873
// visible for testing

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

+31-14
Original file line numberDiff line numberDiff line change
@@ -1592,23 +1592,15 @@ public IndexCommitRef acquireIndexCommit(final boolean flushFirst) throws Engine
15921592
}
15931593
}
15941594

1595-
@SuppressWarnings("finally")
15961595
private boolean failOnTragicEvent(AlreadyClosedException ex) {
15971596
final boolean engineFailed;
15981597
// if we are already closed due to some tragic exception
15991598
// we need to fail the engine. it might have already been failed before
16001599
// but we are double-checking it's failed and closed
16011600
if (indexWriter.isOpen() == false && indexWriter.getTragicException() != null) {
1602-
if (indexWriter.getTragicException() instanceof Error) {
1603-
try {
1604-
logger.error("tragic event in index writer", ex);
1605-
} finally {
1606-
throw (Error) indexWriter.getTragicException();
1607-
}
1608-
} else {
1609-
failEngine("already closed by tragic event on the index writer", (Exception) indexWriter.getTragicException());
1610-
engineFailed = true;
1611-
}
1601+
maybeDie("tragic event in index writer", indexWriter.getTragicException());
1602+
failEngine("already closed by tragic event on the index writer", (Exception) indexWriter.getTragicException());
1603+
engineFailed = true;
16121604
} else if (translog.isOpen() == false && translog.getTragicException() != null) {
16131605
failEngine("already closed by tragic event on the translog", translog.getTragicException());
16141606
engineFailed = true;
@@ -1929,7 +1921,6 @@ protected void doRun() throws Exception {
19291921

19301922
@Override
19311923
protected void handleMergeException(final Directory dir, final Throwable exc) {
1932-
logger.error("failed to merge", exc);
19331924
engineConfig.getThreadPool().generic().execute(new AbstractRunnable() {
19341925
@Override
19351926
public void onFailure(Exception e) {
@@ -1938,13 +1929,39 @@ public void onFailure(Exception e) {
19381929

19391930
@Override
19401931
protected void doRun() throws Exception {
1941-
MergePolicy.MergeException e = new MergePolicy.MergeException(exc, dir);
1942-
failEngine("merge failed", e);
1932+
/*
1933+
* We do this on another thread rather than the merge thread that we are initially called on so that we have complete
1934+
* confidence that the call stack does not contain catch statements that would cause the error that might be thrown
1935+
* here from being caught and never reaching the uncaught exception handler.
1936+
*/
1937+
maybeDie("fatal error while merging", exc);
1938+
logger.error("failed to merge", exc);
1939+
failEngine("merge failed", new MergePolicy.MergeException(exc, dir));
19431940
}
19441941
});
19451942
}
19461943
}
19471944

1945+
/**
1946+
* If the specified throwable is a fatal error, this throwable will be thrown. Callers should ensure that there are no catch statements
1947+
* that would catch an error in the stack as the fatal error here should go uncaught and be handled by the uncaught exception handler
1948+
* that we install during bootstrap. If the specified throwable is indeed a fatal error, the specified message will attempt to be logged
1949+
* before throwing the fatal error. If the specified throwable is not a fatal error, this method is a no-op.
1950+
*
1951+
* @param maybeMessage the message to maybe log
1952+
* @param maybeFatal the throwable that is maybe fatal
1953+
*/
1954+
@SuppressWarnings("finally")
1955+
private void maybeDie(final String maybeMessage, final Throwable maybeFatal) {
1956+
if (maybeFatal instanceof Error) {
1957+
try {
1958+
logger.error(maybeMessage, maybeFatal);
1959+
} finally {
1960+
throw (Error) maybeFatal;
1961+
}
1962+
}
1963+
}
1964+
19481965
/**
19491966
* Commits the specified index writer.
19501967
*

core/src/test/java/org/elasticsearch/bootstrap/ElasticsearchUncaughtExceptionHandlerTests.java

-2
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,6 @@
1919

2020
package org.elasticsearch.bootstrap;
2121

22-
import org.apache.lucene.index.MergePolicy;
2322
import org.elasticsearch.test.ESTestCase;
2423
import org.junit.Before;
2524

@@ -131,7 +130,6 @@ void onNonFatalUncaught(String threadName, Throwable t) {
131130
}
132131

133132
public void testIsFatalCause() {
134-
assertFatal(new MergePolicy.MergeException(new OutOfMemoryError(), null));
135133
assertFatal(new OutOfMemoryError());
136134
assertFatal(new StackOverflowError());
137135
assertFatal(new InternalError());

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

+1-1
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,7 @@
3030
import java.util.Collections;
3131
import java.util.List;
3232

33-
import static org.elasticsearch.index.translog.TranslogDeletionPolicyTests.createTranslogDeletionPolicy;
33+
import static org.elasticsearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy;
3434
import static org.mockito.Mockito.mock;
3535
import static org.mockito.Mockito.times;
3636
import static org.mockito.Mockito.verify;

0 commit comments

Comments
 (0)