IndexWriter: Treat java.lang.Error as tragedy (#13277)

Background:
Historically IndexWriter treated OutOfMemoryError special, for defensive
reasons. It was expanded to VirtualMachineError, to try to play it safe
in similar disastrous circumstances.

We should treat any Error as a tragedy, as it isn't an Exception, and it
isn't something a "reasonable" application should catch. IndexWriter
should be reasonable. See #7049 for some of the reasoning.

We can't pretend this will detect any possible scenario that might cause
harm, e.g. a jvm bug might simply miscompile some code and cause silent
corruption. But we should try harder by playing by the rules.

Closes #13275
Closes #7049
This commit is contained in:
Robert Muir 2024-04-23 21:40:22 -04:00 committed by GitHub
parent fb12e09ab5
commit 9af3ef8952
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
4 changed files with 64 additions and 34 deletions

View File

@ -183,6 +183,8 @@ Changes in Runtime Behavior
* GITHUB#13293: ConcurrentMergeScheduler now allows up to 50% of the threads of the host to be used
for merging. (Adrien Grand)
* GITHUB#13277: IndexWriter treats any java.lang.Error as tragic. (Robert Muir)
Changes in Backwards Compatibility Policy
-----------------------------------------

View File

@ -153,9 +153,9 @@ import org.apache.lucene.util.Version;
* and it decides when and how to run the merges. The default is {@link ConcurrentMergeScheduler}.
* <a id="OOME"></a>
*
* <p><b>NOTE</b>: if you hit a VirtualMachineError, or disaster strikes during a checkpoint then
* IndexWriter will close itself. This is a defensive measure in case any internal state (buffered
* documents, deletions, reference counts) were corrupted. Any subsequent calls will throw an
* <p><b>NOTE</b>: if you hit an Error, or disaster strikes during a checkpoint then IndexWriter
* will close itself. This is a defensive measure in case any internal state (buffered documents,
* deletions, reference counts) were corrupted. Any subsequent calls will throw an
* AlreadyClosedException. <a id="thread-safety"></a>
*
* <p><b>NOTE</b>: {@link IndexWriter} instances are completely thread safe, meaning multiple
@ -704,7 +704,7 @@ public class IndexWriter
infoStream.message("IW", "getReader took " + (System.currentTimeMillis() - tStart) + " ms");
}
success2 = true;
} catch (VirtualMachineError tragedy) {
} catch (Error tragedy) {
tragicEvent(tragedy, "getReader");
throw tragedy;
} finally {
@ -1549,7 +1549,7 @@ public class IndexWriter
final long seqNo = maybeProcessEvents(docWriter.updateDocuments(docs, delNode));
success = true;
return seqNo;
} catch (VirtualMachineError tragedy) {
} catch (Error tragedy) {
tragicEvent(tragedy, "updateDocuments");
throw tragedy;
} finally {
@ -1786,7 +1786,7 @@ public class IndexWriter
ensureOpen();
try {
return maybeProcessEvents(docWriter.deleteTerms(terms));
} catch (VirtualMachineError tragedy) {
} catch (Error tragedy) {
tragicEvent(tragedy, "deleteDocuments(Term..)");
throw tragedy;
}
@ -1813,7 +1813,7 @@ public class IndexWriter
try {
return maybeProcessEvents(docWriter.deleteQueries(queries));
} catch (VirtualMachineError tragedy) {
} catch (Error tragedy) {
tragicEvent(tragedy, "deleteDocuments(Query..)");
throw tragedy;
}
@ -1892,7 +1892,7 @@ public class IndexWriter
try {
return maybeProcessEvents(
docWriter.updateDocValues(new NumericDocValuesUpdate(term, field, value)));
} catch (VirtualMachineError tragedy) {
} catch (Error tragedy) {
tragicEvent(tragedy, "updateNumericDocValue");
throw tragedy;
}
@ -1922,7 +1922,7 @@ public class IndexWriter
try {
return maybeProcessEvents(
docWriter.updateDocValues(new BinaryDocValuesUpdate(term, field, value)));
} catch (VirtualMachineError tragedy) {
} catch (Error tragedy) {
tragicEvent(tragedy, "updateBinaryDocValue");
throw tragedy;
}
@ -1944,7 +1944,7 @@ public class IndexWriter
DocValuesUpdate[] dvUpdates = buildDocValuesUpdate(term, updates);
try {
return maybeProcessEvents(docWriter.updateDocValues(dvUpdates));
} catch (VirtualMachineError tragedy) {
} catch (Error tragedy) {
tragicEvent(tragedy, "updateDocValues");
throw tragedy;
}
@ -2577,7 +2577,7 @@ public class IndexWriter
} catch (Throwable t) {
throwable.addSuppressed(t);
} finally {
if (throwable instanceof VirtualMachineError) {
if (throwable instanceof Error) {
try {
tragicEvent(throwable, "rollbackInternal");
} catch (Throwable t1) {
@ -2678,7 +2678,7 @@ public class IndexWriter
}
}
}
} catch (VirtualMachineError tragedy) {
} catch (Error tragedy) {
tragicEvent(tragedy, "deleteAll");
throw tragedy;
}
@ -3098,7 +3098,7 @@ public class IndexWriter
successTop = true;
} catch (VirtualMachineError tragedy) {
} catch (Error tragedy) {
tragicEvent(tragedy, "addIndexes(Directory...)");
throw tragedy;
} finally {
@ -3270,7 +3270,7 @@ public class IndexWriter
throw new RuntimeException(
"failed to successfully merge all provided readers in addIndexes(CodecReader...)");
}
} catch (VirtualMachineError tragedy) {
} catch (Error tragedy) {
tragicEvent(tragedy, "addIndexes(CodecReader...)");
throw tragedy;
}
@ -3623,7 +3623,7 @@ public class IndexWriter
return true; // we wrote a segment
}
return false;
} catch (VirtualMachineError tragedy) {
} catch (Error tragedy) {
tragicEvent(tragedy, "flushNextBuffer");
throw tragedy;
} finally {
@ -3739,7 +3739,7 @@ public class IndexWriter
doAfterFlush();
}
}
} catch (VirtualMachineError tragedy) {
} catch (Error tragedy) {
tragicEvent(tragedy, "prepareCommit");
throw tragedy;
} finally {
@ -4299,7 +4299,7 @@ public class IndexWriter
success = true;
return anyChanges;
}
} catch (VirtualMachineError tragedy) {
} catch (Error tragedy) {
tragicEvent(tragedy, "doFlush");
throw tragedy;
} finally {
@ -5693,7 +5693,7 @@ public class IndexWriter
segmentInfos.updateGeneration(toSync);
}
}
} catch (VirtualMachineError tragedy) {
} catch (Error tragedy) {
tragicEvent(tragedy, "startCommit");
throw tragedy;
}

View File

@ -105,21 +105,20 @@ public final class IOUtils {
}
/**
* Closes all given <code>Closeable</code>s, suppressing all thrown non {@link
* VirtualMachineError} exceptions. Even if a {@link VirtualMachineError} is thrown all given
* closeable are closed.
* Closes all given <code>Closeable</code>s, suppressing all thrown non {@link Error} exceptions.
* Even if a {@link Error} is thrown all given closeable are closed.
*
* @see #closeWhileHandlingException(Closeable...)
*/
public static void closeWhileHandlingException(Iterable<? extends Closeable> objects) {
VirtualMachineError firstError = null;
Error firstError = null;
Throwable firstThrowable = null;
for (Closeable object : objects) {
try {
if (object != null) {
object.close();
}
} catch (VirtualMachineError e) {
} catch (Error e) {
firstError = useOrSuppress(firstError, e);
} catch (Throwable t) {
firstThrowable = useOrSuppress(firstThrowable, t);
@ -128,7 +127,7 @@ public final class IOUtils {
if (firstError != null) {
// we ensure that we bubble up any errors. We can't recover from these but need to make sure
// they are
// bubbled up. if a non-VMError is thrown we also add the suppressed exceptions to it.
// bubbled up. if a non-Error is thrown we also add the suppressed exceptions to it.
if (firstThrowable != null) {
firstError.addSuppressed(firstThrowable);
}

View File

@ -19,6 +19,7 @@ package org.apache.lucene.index;
import static java.nio.charset.StandardCharsets.UTF_8;
import java.io.ByteArrayOutputStream;
import java.io.IOError;
import java.io.IOException;
import java.io.PrintStream;
import java.util.Arrays;
@ -53,7 +54,7 @@ import org.apache.lucene.util.IOUtils;
* index corruption is ever created.
*/
@SuppressCodecs("SimpleText")
public class TestIndexWriterOnVMError extends LuceneTestCase {
public class TestIndexWriterOnError extends LuceneTestCase {
// just one thread, serial merge policy, hopefully debuggable
private void doTest(MockDirectoryWrapper.Failure failOn) throws Exception {
@ -151,7 +152,7 @@ public class TestIndexWriterOnVMError extends LuceneTestCase {
"dv2",
new BytesRef(Integer.toString(i + 1)));
}
} catch (VirtualMachineError | AlreadyClosedException disaster) {
} catch (Error | AlreadyClosedException disaster) {
getTragedy(disaster, iw, exceptionStream);
continue STARTOVER;
}
@ -174,7 +175,7 @@ public class TestIndexWriterOnVMError extends LuceneTestCase {
iw.deleteDocuments(
new Term("id", Integer.toString(i)), new Term("id", Integer.toString(-i)));
}
} catch (VirtualMachineError | AlreadyClosedException disaster) {
} catch (Error | AlreadyClosedException disaster) {
getTragedy(disaster, iw, exceptionStream);
continue STARTOVER;
}
@ -197,7 +198,7 @@ public class TestIndexWriterOnVMError extends LuceneTestCase {
if (DirectoryReader.indexExists(dir)) {
TestUtil.checkIndex(dir);
}
} catch (VirtualMachineError | AlreadyClosedException disaster) {
} catch (Error | AlreadyClosedException disaster) {
getTragedy(disaster, iw, exceptionStream);
continue STARTOVER;
}
@ -206,7 +207,7 @@ public class TestIndexWriterOnVMError extends LuceneTestCase {
try {
iw.close();
} catch (VirtualMachineError | AlreadyClosedException disaster) {
} catch (Error | AlreadyClosedException disaster) {
getTragedy(disaster, iw, exceptionStream);
continue STARTOVER;
}
@ -225,15 +226,13 @@ public class TestIndexWriterOnVMError extends LuceneTestCase {
}
}
private VirtualMachineError getTragedy(Throwable disaster, IndexWriter writer, PrintStream log) {
private Error getTragedy(Throwable disaster, IndexWriter writer, PrintStream log) {
Throwable e = disaster;
if (e instanceof AlreadyClosedException) {
e = e.getCause();
}
if (e instanceof VirtualMachineError
&& e.getMessage() != null
&& e.getMessage().startsWith("Fake")) {
if (e instanceof Error && e.getMessage() != null && e.getMessage().contains("Fake")) {
log.println("\nTEST: got expected fake exc:" + e.getMessage());
e.printStackTrace(log);
// TODO: remove rollback here, and add this assert to ensure "full OOM protection" anywhere IW
@ -244,7 +243,7 @@ public class TestIndexWriterOnVMError extends LuceneTestCase {
} catch (Throwable t) {
t.printStackTrace(log);
}
return (VirtualMachineError) e;
return (Error) e;
} else {
Rethrow.rethrow(disaster);
return null; // dead
@ -281,6 +280,36 @@ public class TestIndexWriterOnVMError extends LuceneTestCase {
});
}
public void testLinkageError() throws Exception {
final Random r = new Random(random().nextLong());
doTest(
new Failure() {
@Override
public void eval(MockDirectoryWrapper dir) throws IOException {
if (r.nextInt(3000) == 0) {
if (callStackContains(IndexWriter.class)) {
throw new LinkageError("Fake LinkageError");
}
}
}
});
}
public void testIOError() throws Exception {
final Random r = new Random(random().nextLong());
doTest(
new Failure() {
@Override
public void eval(MockDirectoryWrapper dir) throws IOException {
if (r.nextInt(3000) == 0) {
if (callStackContains(IndexWriter.class)) {
throw new IOError(new RuntimeException("Fake IOError"));
}
}
}
});
}
@Nightly
public void testCheckpoint() throws Exception {
final Random r = new Random(random().nextLong());