LUCENE-6872: IndexWriter OOM handling should be any VirtualMachineError

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1712310 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2015-11-03 15:29:02 +00:00
parent 724e803e19
commit 69b467a24b
3 changed files with 65 additions and 42 deletions

View File

@ -239,6 +239,9 @@ Bug Fixes
* LUCENE-6858: Fix ContextSuggestField to correctly wrap token stream
when using CompletionAnalyzer. (Areek Zillur)
* LUCENE-6872: IndexWriter handles any VirtualMachineError, not just OOM,
as tragic. (Robert Muir)
Other
* LUCENE-6478: Test execution can hang with java.security.debug. (Dawid Weiss)

View File

@ -149,8 +149,8 @@ import org.apache.lucene.util.Version;
it decides when and how to run the merges. The default is
{@link ConcurrentMergeScheduler}. </p>
<a name="OOME"></a><p><b>NOTE</b>: if you hit an
OutOfMemoryError, or disaster strikes during a checkpoint
<a name="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.
@ -456,7 +456,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
infoStream.message("IW", "getReader took " + (System.currentTimeMillis() - tStart) + " msec");
}
success2 = true;
} catch (AbortingException | OutOfMemoryError tragedy) {
} catch (AbortingException | VirtualMachineError tragedy) {
tragicEvent(tragedy, "getReader");
// never reached but javac disagrees:
return null;
@ -1310,7 +1310,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
}
}
} catch (AbortingException | OutOfMemoryError tragedy) {
} catch (AbortingException | VirtualMachineError tragedy) {
tragicEvent(tragedy, "updateDocuments");
}
}
@ -1406,8 +1406,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
if (docWriter.deleteTerms(terms)) {
processEvents(true, false);
}
} catch (OutOfMemoryError oom) {
tragicEvent(oom, "deleteDocuments(Term..)");
} catch (VirtualMachineError tragedy) {
tragicEvent(tragedy, "deleteDocuments(Term..)");
}
}
@ -1435,8 +1435,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
if (docWriter.deleteQueries(queries)) {
processEvents(true, false);
}
} catch (OutOfMemoryError oom) {
tragicEvent(oom, "deleteDocuments(Query..)");
} catch (VirtualMachineError tragedy) {
tragicEvent(tragedy, "deleteDocuments(Query..)");
}
}
@ -1469,7 +1469,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
}
}
} catch (AbortingException | OutOfMemoryError tragedy) {
} catch (AbortingException | VirtualMachineError tragedy) {
tragicEvent(tragedy, "updateDocument");
}
}
@ -1499,8 +1499,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
if (docWriter.updateDocValues(new NumericDocValuesUpdate(term, field, value))) {
processEvents(true, false);
}
} catch (OutOfMemoryError oom) {
tragicEvent(oom, "updateNumericDocValue");
} catch (VirtualMachineError tragedy) {
tragicEvent(tragedy, "updateNumericDocValue");
}
}
@ -1536,8 +1536,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
if (docWriter.updateDocValues(new BinaryDocValuesUpdate(term, field, value))) {
processEvents(true, false);
}
} catch (OutOfMemoryError oom) {
tragicEvent(oom, "updateBinaryDocValue");
} catch (VirtualMachineError tragedy) {
tragicEvent(tragedy, "updateBinaryDocValue");
}
}
@ -1584,8 +1584,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
if (docWriter.updateDocValues(dvUpdates)) {
processEvents(true, false);
}
} catch (OutOfMemoryError oom) {
tragicEvent(oom, "updateDocValues");
} catch (VirtualMachineError tragedy) {
tragicEvent(tragedy, "updateDocValues");
}
}
@ -2095,8 +2095,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
success = true;
} catch (OutOfMemoryError oom) {
tragicEvent(oom, "rollbackInternal");
} catch (VirtualMachineError tragedy) {
tragicEvent(tragedy, "rollbackInternal");
} finally {
if (success == false) {
// Must not hold IW's lock while closing
@ -2215,8 +2215,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
}
}
} catch (OutOfMemoryError oom) {
tragicEvent(oom, "deleteAll");
} catch (VirtualMachineError tragedy) {
tragicEvent(tragedy, "deleteAll");
}
}
@ -2531,8 +2531,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
successTop = true;
} catch (OutOfMemoryError oom) {
tragicEvent(oom, "addIndexes(Directory...)");
} catch (VirtualMachineError tragedy) {
tragicEvent(tragedy, "addIndexes(Directory...)");
} finally {
if (successTop) {
IOUtils.close(locks);
@ -2671,8 +2671,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
segmentInfos.add(infoPerCommit);
checkpoint();
}
} catch (OutOfMemoryError oom) {
tragicEvent(oom, "addIndexes(CodecReader...)");
} catch (VirtualMachineError tragedy) {
tragicEvent(tragedy, "addIndexes(CodecReader...)");
}
maybeMerge();
}
@ -2833,7 +2833,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
doAfterFlush();
}
}
} catch (AbortingException | OutOfMemoryError tragedy) {
} catch (AbortingException | VirtualMachineError tragedy) {
tragicEvent(tragedy, "prepareCommit");
}
@ -3106,7 +3106,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
success = true;
return anyChanges;
}
} catch (AbortingException | OutOfMemoryError tragedy) {
} catch (AbortingException | VirtualMachineError tragedy) {
tragicEvent(tragedy, "doFlush");
// never hit
return false;
@ -4447,8 +4447,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
}
}
} catch (OutOfMemoryError oom) {
tragicEvent(oom, "startCommit");
} catch (VirtualMachineError tragedy) {
tragicEvent(tragedy, "startCommit");
}
testPoint("finishStartCommit");
}

View File

@ -43,17 +43,16 @@ import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.LuceneTestCase.Nightly;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.apache.lucene.util.Rethrow;
import org.apache.lucene.util.TestUtil;
/**
* Causes a bunch of fake OOM and checks that no other exceptions are delivered instead,
* Causes a bunch of fake VM errors and checks that no other exceptions are delivered instead,
* no index corruption is ever created.
*/
@SuppressCodecs("SimpleText")
public class TestIndexWriterOutOfMemory extends LuceneTestCase {
public class TestIndexWriterOnVMError extends LuceneTestCase {
// just one thread, serial merge policy, hopefully debuggable
private void doTest(MockDirectoryWrapper.Failure failOn) throws Exception {
@ -139,8 +138,8 @@ public class TestIndexWriterOutOfMemory extends LuceneTestCase {
} else if (thingToDo == 2) {
iw.updateBinaryDocValue(new Term("id", Integer.toString(i)), "dv2", new BytesRef(Integer.toString(i+1)));
}
} catch (OutOfMemoryError | AlreadyClosedException disaster) {
getOOM(disaster, iw, exceptionStream);
} catch (VirtualMachineError | AlreadyClosedException disaster) {
getTragedy(disaster, iw, exceptionStream);
continue STARTOVER;
}
} else {
@ -158,8 +157,8 @@ public class TestIndexWriterOutOfMemory extends LuceneTestCase {
if (random().nextBoolean()) {
iw.deleteDocuments(new Term("id", Integer.toString(i)), new Term("id", Integer.toString(-i)));
}
} catch (OutOfMemoryError | AlreadyClosedException disaster) {
getOOM(disaster, iw, exceptionStream);
} catch (VirtualMachineError | AlreadyClosedException disaster) {
getTragedy(disaster, iw, exceptionStream);
continue STARTOVER;
}
}
@ -181,8 +180,8 @@ public class TestIndexWriterOutOfMemory extends LuceneTestCase {
if (DirectoryReader.indexExists(dir)) {
TestUtil.checkIndex(dir);
}
} catch (OutOfMemoryError | AlreadyClosedException disaster) {
getOOM(disaster, iw, exceptionStream);
} catch (VirtualMachineError | AlreadyClosedException disaster) {
getTragedy(disaster, iw, exceptionStream);
continue STARTOVER;
}
}
@ -190,8 +189,8 @@ public class TestIndexWriterOutOfMemory extends LuceneTestCase {
try {
iw.close();
} catch (OutOfMemoryError | AlreadyClosedException disaster) {
getOOM(disaster, iw, exceptionStream);
} catch (VirtualMachineError | AlreadyClosedException disaster) {
getTragedy(disaster, iw, exceptionStream);
continue STARTOVER;
}
} catch (Throwable t) {
@ -209,13 +208,13 @@ public class TestIndexWriterOutOfMemory extends LuceneTestCase {
}
}
private OutOfMemoryError getOOM(Throwable disaster, IndexWriter writer, PrintStream log) {
private VirtualMachineError getTragedy(Throwable disaster, IndexWriter writer, PrintStream log) {
Throwable e = disaster;
if (e instanceof AlreadyClosedException) {
e = e.getCause();
}
if (e instanceof OutOfMemoryError && e.getMessage() != null && e.getMessage().startsWith("Fake OutOfMemoryError")) {
if (e instanceof VirtualMachineError && e.getMessage() != null && e.getMessage().startsWith("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 does writes
@ -223,14 +222,14 @@ public class TestIndexWriterOutOfMemory extends LuceneTestCase {
try {
writer.rollback();
} catch (Throwable t) {}
return (OutOfMemoryError) e;
return (VirtualMachineError) e;
} else {
Rethrow.rethrow(disaster);
return null; // dead
}
}
public void testBasics() throws Exception {
public void testOOM() throws Exception {
final Random r = new Random(random().nextLong());
doTest(new Failure() {
@Override
@ -251,6 +250,27 @@ public class TestIndexWriterOutOfMemory extends LuceneTestCase {
});
}
public void testUnknownError() 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) {
StackTraceElement stack[] = Thread.currentThread().getStackTrace();
boolean ok = false;
for (int i = 0; i < stack.length; i++) {
if (stack[i].getClassName().equals(IndexWriter.class.getName())) {
ok = true;
}
}
if (ok) {
throw new UnknownError("Fake UnknownError");
}
}
}
});
}
@Nightly
public void testCheckpoint() throws Exception {
final Random r = new Random(random().nextLong());