mirror of https://github.com/apache/lucene.git
LUCENE-5958: OOM or exceptions during checkpoint make IndexWriter have a bad day
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1625853 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
bcf58db693
commit
7a876d8865
lucene
|
@ -169,6 +169,11 @@ Bug Fixes
|
|||
* LUCENE-5934: Fix backwards compatibility for 4.0 indexes.
|
||||
(Ian Lea, Uwe Schindler, Robert Muir, Ryan Ernst)
|
||||
|
||||
* LUCENE-5958: Don't let exceptions during checkpoint corrupt the index.
|
||||
Refactor existing OOM handling too, so you don't need to handle OOM special
|
||||
for every IndexWriter method: instead such disasters will cause IW to close itself
|
||||
defensively. (Robert Muir, Mike McCandless)
|
||||
|
||||
Tests
|
||||
|
||||
* LUCENE-5936: Add backcompat checks to verify what is tested matches known versions
|
||||
|
|
|
@ -27,6 +27,7 @@ import java.util.Collections;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.regex.Matcher;
|
||||
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
|
@ -109,7 +110,6 @@ final class IndexFileDeleter implements Closeable {
|
|||
* infoStream is enabled */
|
||||
public static boolean VERBOSE_REF_COUNTS = false;
|
||||
|
||||
// Used only for assert
|
||||
private final IndexWriter writer;
|
||||
|
||||
// called only from assert
|
||||
|
@ -126,6 +126,7 @@ final class IndexFileDeleter implements Closeable {
|
|||
*/
|
||||
public IndexFileDeleter(Directory directory, IndexDeletionPolicy policy, SegmentInfos segmentInfos,
|
||||
InfoStream infoStream, IndexWriter writer, boolean initialIndexExists) throws IOException {
|
||||
Objects.requireNonNull(writer);
|
||||
this.infoStream = infoStream;
|
||||
this.writer = writer;
|
||||
|
||||
|
@ -343,10 +344,10 @@ final class IndexFileDeleter implements Closeable {
|
|||
}
|
||||
|
||||
private void ensureOpen() throws AlreadyClosedException {
|
||||
if (writer == null) {
|
||||
throw new AlreadyClosedException("this IndexWriter is closed");
|
||||
} else {
|
||||
writer.ensureOpen(false);
|
||||
writer.ensureOpen(false);
|
||||
// since we allow 'closing' state, we must still check this, we could be closing because we hit e.g. OOM
|
||||
if (writer.tragedy != null) {
|
||||
throw new AlreadyClosedException("refusing to delete any files: this IndexWriter hit an unrecoverable exception", writer.tragedy);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -147,16 +147,11 @@ import org.apache.lucene.util.Version;
|
|||
{@link ConcurrentMergeScheduler}. </p>
|
||||
|
||||
<a name="OOME"></a><p><b>NOTE</b>: if you hit an
|
||||
OutOfMemoryError then IndexWriter will quietly record this
|
||||
fact and block all future segment commits. This is a
|
||||
OutOfMemoryError, or disaster strikes during a checkpoint
|
||||
then IndexWriter will close itself. This is a
|
||||
defensive measure in case any internal state (buffered
|
||||
documents and deletions) were corrupted. Any subsequent
|
||||
calls to {@link #commit()} will throw an
|
||||
IllegalStateException. The only course of action is to
|
||||
call {@link #close()}, which internally will call {@link
|
||||
#rollback()}, to undo any changes to the index since the
|
||||
last commit. You can also just call {@link #rollback()}
|
||||
directly.</p>
|
||||
documents, deletions, reference counts) were corrupted.
|
||||
Any subsequent calls will throw an AlreadyClosedException.</p>
|
||||
|
||||
<a name="thread-safety"></a><p><b>NOTE</b>: {@link
|
||||
IndexWriter} instances are completely thread
|
||||
|
@ -246,7 +241,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* IndexWriterConfig#setInfoStream(InfoStream)}).
|
||||
*/
|
||||
public final static int MAX_TERM_LENGTH = DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8;
|
||||
volatile private boolean hitOOM;
|
||||
// when unrecoverable disaster strikes, we populate this with the reason that we had to close IndexWriter
|
||||
volatile Throwable tragedy;
|
||||
|
||||
private final Directory directory; // where this index resides
|
||||
private final Analyzer analyzer; // how to analyze text
|
||||
|
@ -428,7 +424,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "getReader");
|
||||
tragicEvent(oom, "getReader");
|
||||
// never reached but javac disagrees:
|
||||
return null;
|
||||
} finally {
|
||||
|
@ -437,10 +433,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
infoStream.message("IW", "hit exception during NRT reader");
|
||||
}
|
||||
}
|
||||
// Done: finish the full flush!
|
||||
docWriter.finishFullFlush(success);
|
||||
processEvents(false, true);
|
||||
doAfterFlush();
|
||||
if (tragedy == null) {
|
||||
// Done: finish the full flush! (unless we hit OOM or something)
|
||||
docWriter.finishFullFlush(success);
|
||||
processEvents(false, true);
|
||||
doAfterFlush();
|
||||
}
|
||||
}
|
||||
}
|
||||
if (anySegmentFlushed) {
|
||||
|
@ -702,7 +700,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
*/
|
||||
protected final void ensureOpen(boolean failIfClosing) throws AlreadyClosedException {
|
||||
if (closed || (failIfClosing && closing)) {
|
||||
throw new AlreadyClosedException("this IndexWriter is closed");
|
||||
throw new AlreadyClosedException("this IndexWriter is closed", tragedy);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1081,10 +1079,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* replaced with the Unicode replacement character
|
||||
* U+FFFD.</p>
|
||||
*
|
||||
* <p><b>NOTE</b>: if this method hits an OutOfMemoryError
|
||||
* you should immediately close the writer. See <a
|
||||
* href="#OOME">above</a> for details.</p>
|
||||
*
|
||||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
|
@ -1100,10 +1094,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* index and IndexWriter state after an Exception, and
|
||||
* flushing/merging temporary free space requirements.</p>
|
||||
*
|
||||
* <p><b>NOTE</b>: if this method hits an OutOfMemoryError
|
||||
* you should immediately close the writer. See <a
|
||||
* href="#OOME">above</a> for details.</p>
|
||||
*
|
||||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
|
@ -1143,10 +1133,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* and will likely break them up. Use such tools at your
|
||||
* own risk!
|
||||
*
|
||||
* <p><b>NOTE</b>: if this method hits an OutOfMemoryError
|
||||
* you should immediately close the writer. See <a
|
||||
* href="#OOME">above</a> for details.</p>
|
||||
*
|
||||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*
|
||||
|
@ -1219,7 +1205,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "updateDocuments");
|
||||
tragicEvent(oom, "updateDocuments");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1303,10 +1289,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* terms. All given deletes are applied and flushed atomically
|
||||
* at the same time.
|
||||
*
|
||||
* <p><b>NOTE</b>: if this method hits an OutOfMemoryError
|
||||
* you should immediately close the writer. See <a
|
||||
* href="#OOME">above</a> for details.</p>
|
||||
*
|
||||
* @param terms array of terms to identify the documents
|
||||
* to be deleted
|
||||
* @throws CorruptIndexException if the index is corrupt
|
||||
|
@ -1319,7 +1301,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
processEvents(true, false);
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "deleteDocuments(Term..)");
|
||||
tragicEvent(oom, "deleteDocuments(Term..)");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1327,10 +1309,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* Deletes the document(s) matching any of the provided queries.
|
||||
* All given deletes are applied and flushed atomically at the same time.
|
||||
*
|
||||
* <p><b>NOTE</b>: if this method hits an OutOfMemoryError
|
||||
* you should immediately close the writer. See <a
|
||||
* href="#OOME">above</a> for details.</p>
|
||||
*
|
||||
* @param queries array of queries to identify the documents
|
||||
* to be deleted
|
||||
* @throws CorruptIndexException if the index is corrupt
|
||||
|
@ -1343,7 +1321,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
processEvents(true, false);
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "deleteDocuments(Query..)");
|
||||
tragicEvent(oom, "deleteDocuments(Query..)");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1354,10 +1332,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* by a reader on the same index (flush may happen only after
|
||||
* the add).
|
||||
*
|
||||
* <p><b>NOTE</b>: if this method hits an OutOfMemoryError
|
||||
* you should immediately close the writer. See <a
|
||||
* href="#OOME">above</a> for details.</p>
|
||||
*
|
||||
* @param term the term to identify the document(s) to be
|
||||
* deleted
|
||||
* @param doc the document to be added
|
||||
|
@ -1376,10 +1350,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* by a reader on the same index (flush may happen only after
|
||||
* the add).
|
||||
*
|
||||
* <p><b>NOTE</b>: if this method hits an OutOfMemoryError
|
||||
* you should immediately close the writer. See <a
|
||||
* href="#OOME">above</a> for details.</p>
|
||||
*
|
||||
* @param term the term to identify the document(s) to be
|
||||
* deleted
|
||||
* @param doc the document to be added
|
||||
|
@ -1405,7 +1375,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "updateDocument");
|
||||
tragicEvent(oom, "updateDocument");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1414,11 +1384,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* given <code>value</code>. You can only update fields that already exist in
|
||||
* the index, not add new fields through this method.
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
|
||||
* close the writer. See <a href="#OOME">above</a> for details.
|
||||
* </p>
|
||||
*
|
||||
* @param term
|
||||
* the term to identify the document(s) to be updated
|
||||
* @param field
|
||||
|
@ -1440,7 +1405,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
processEvents(true, false);
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "updateNumericDocValue");
|
||||
tragicEvent(oom, "updateNumericDocValue");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1453,11 +1418,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* <b>NOTE:</b> this method currently replaces the existing value of all
|
||||
* affected documents with the new value.
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE:</b> if this method hits an OutOfMemoryError you should immediately
|
||||
* close the writer. See <a href="#OOME">above</a> for details.
|
||||
* </p>
|
||||
*
|
||||
* @param term
|
||||
* the term to identify the document(s) to be updated
|
||||
* @param field
|
||||
|
@ -1482,7 +1442,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
processEvents(true, false);
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "updateBinaryDocValue");
|
||||
tragicEvent(oom, "updateBinaryDocValue");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1492,11 +1452,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* {@link Term} to the same value. All updates are atomically applied and
|
||||
* flushed together.
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
|
||||
* close the writer. See <a href="#OOME">above</a> for details.
|
||||
* </p>
|
||||
*
|
||||
* @param updates
|
||||
* the updates to apply
|
||||
* @throws CorruptIndexException
|
||||
|
@ -1532,7 +1487,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
processEvents(true, false);
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "updateDocValues");
|
||||
tragicEvent(oom, "updateDocValues");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1637,10 +1592,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* newly created segments will not be merged unless you
|
||||
* call forceMerge again.</p>
|
||||
*
|
||||
* <p><b>NOTE</b>: if this method hits an OutOfMemoryError
|
||||
* you should immediately close the writer. See <a
|
||||
* href="#OOME">above</a> for details.</p>
|
||||
*
|
||||
* <p><b>NOTE</b>: if you call {@link #abortMerges}, which
|
||||
* aborts all running merges, then any thread still
|
||||
* running this method might hit a {@link
|
||||
|
@ -1663,10 +1614,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* all merging completes. This is only meaningful with a
|
||||
* {@link MergeScheduler} that is able to run merges in
|
||||
* background threads.
|
||||
*
|
||||
* <p><b>NOTE</b>: if this method hits an OutOfMemoryError
|
||||
* you should immediately close the writer. See <a
|
||||
* href="#OOME">above</a> for details.</p>
|
||||
*/
|
||||
public void forceMerge(int maxNumSegments, boolean doWait) throws IOException {
|
||||
ensureOpen();
|
||||
|
@ -1708,8 +1655,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
synchronized(this) {
|
||||
while(true) {
|
||||
|
||||
if (hitOOM) {
|
||||
throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot complete forceMerge");
|
||||
if (tragedy != null) {
|
||||
throw new IllegalStateException("this writer hit an unrecoverable error; cannot complete forceMerge", tragedy);
|
||||
}
|
||||
|
||||
if (mergeExceptions.size() > 0) {
|
||||
|
@ -1764,10 +1711,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* {@link MergeScheduler} that is able to run merges in
|
||||
* background threads.
|
||||
*
|
||||
* <p><b>NOTE</b>: if this method hits an OutOfMemoryError
|
||||
* you should immediately close the writer. See <a
|
||||
* href="#OOME">above</a> for details.</p>
|
||||
*
|
||||
* <p><b>NOTE</b>: if you call {@link #abortMerges}, which
|
||||
* aborts all running merges, then any thread still
|
||||
* running this method might hit a {@link
|
||||
|
@ -1803,8 +1746,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
boolean running = true;
|
||||
while(running) {
|
||||
|
||||
if (hitOOM) {
|
||||
throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot complete forceMergeDeletes");
|
||||
if (tragedy != null) {
|
||||
throw new IllegalStateException("this writer hit an unrecoverable error; cannot complete forceMergeDeletes", tragedy);
|
||||
}
|
||||
|
||||
// Check each merge that MergePolicy asked us to
|
||||
|
@ -1853,10 +1796,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* <p><b>NOTE</b>: this method first flushes a new
|
||||
* segment (if there are indexed documents), and applies
|
||||
* all buffered deletes.
|
||||
*
|
||||
* <p><b>NOTE</b>: if this method hits an OutOfMemoryError
|
||||
* you should immediately close the writer. See <a
|
||||
* href="#OOME">above</a> for details.</p>
|
||||
*/
|
||||
public void forceMergeDeletes() throws IOException {
|
||||
forceMergeDeletes(true);
|
||||
|
@ -1874,10 +1813,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
*
|
||||
* This method will call the {@link MergePolicy} with
|
||||
* {@link MergeTrigger#EXPLICIT}.
|
||||
*
|
||||
* <p><b>NOTE</b>: if this method hits an OutOfMemoryError
|
||||
* you should immediately close the writer. See <a
|
||||
* href="#OOME">above</a> for details.</p>
|
||||
*/
|
||||
public final void maybeMerge() throws IOException {
|
||||
maybeMerge(config.getMergePolicy(), MergeTrigger.EXPLICIT, UNBOUNDED_MAX_MERGE_SEGMENTS);
|
||||
|
@ -1902,8 +1837,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
return false;
|
||||
}
|
||||
|
||||
// Do not start new merges if we've hit OOME
|
||||
if (hitOOM) {
|
||||
// Do not start new merges if disaster struck
|
||||
if (tragedy != null) {
|
||||
return false;
|
||||
}
|
||||
boolean newMergesFound = false;
|
||||
|
@ -2064,7 +1999,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
|
||||
success = true;
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "rollbackInternal");
|
||||
tragicEvent(oom, "rollbackInternal");
|
||||
} finally {
|
||||
if (!success) {
|
||||
// Must not hold IW's lock while closing
|
||||
|
@ -2165,7 +2100,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
globalFieldNumberMap.clear();
|
||||
success = true;
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "deleteAll");
|
||||
tragicEvent(oom, "deleteAll");
|
||||
} finally {
|
||||
if (!success) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
|
@ -2399,11 +2334,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
*
|
||||
* <p>This requires this index not be among those to be added.
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE</b>: if this method hits an OutOfMemoryError
|
||||
* you should immediately close the writer. See <a
|
||||
* href="#OOME">above</a> for details.
|
||||
*
|
||||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
* @throws LockObtainFailedException if we were unable to
|
||||
|
@ -2495,7 +2425,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
successTop = true;
|
||||
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "addIndexes(Directory...)");
|
||||
tragicEvent(oom, "addIndexes(Directory...)");
|
||||
} finally {
|
||||
if (successTop) {
|
||||
IOUtils.close(locks);
|
||||
|
@ -2517,10 +2447,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* free space required in the Directory, and non-CFS segments on an Exception.
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
|
||||
* close the writer. See <a href="#OOME">above</a> for details.
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE:</b> empty segments are dropped by this method and not added to this
|
||||
* index.
|
||||
*
|
||||
|
@ -2657,7 +2583,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
checkpoint();
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "addIndexes(IndexReader...)");
|
||||
tragicEvent(oom, "addIndexes(IndexReader...)");
|
||||
}
|
||||
maybeMerge();
|
||||
}
|
||||
|
@ -2756,10 +2682,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* <p>You can also just call {@link #commit()} directly
|
||||
* without prepareCommit first in which case that method
|
||||
* will internally call prepareCommit.
|
||||
*
|
||||
* <p><b>NOTE</b>: if this method hits an OutOfMemoryError
|
||||
* you should immediately close the writer. See <a
|
||||
* href="#OOME">above</a> for details.</p>
|
||||
*/
|
||||
@Override
|
||||
public final void prepareCommit() throws IOException {
|
||||
|
@ -2776,8 +2698,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
infoStream.message("IW", " index before flush " + segString());
|
||||
}
|
||||
|
||||
if (hitOOM) {
|
||||
throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot commit");
|
||||
if (tragedy != null) {
|
||||
throw new IllegalStateException("this writer hit an unrecoverable error; cannot commit", tragedy);
|
||||
}
|
||||
|
||||
if (pendingCommit != null) {
|
||||
|
@ -2843,7 +2765,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "prepareCommit");
|
||||
tragicEvent(oom, "prepareCommit");
|
||||
}
|
||||
|
||||
boolean success = false;
|
||||
|
@ -2916,10 +2838,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* loss it may still lose data. Lucene cannot guarantee
|
||||
* consistency on such devices. </p>
|
||||
*
|
||||
* <p><b>NOTE</b>: if this method hits an OutOfMemoryError
|
||||
* you should immediately close the writer. See <a
|
||||
* href="#OOME">above</a> for details.</p>
|
||||
*
|
||||
* @see #prepareCommit
|
||||
*/
|
||||
@Override
|
||||
|
@ -2971,25 +2889,43 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
|
||||
private synchronized final void finishCommit() throws IOException {
|
||||
|
||||
boolean success = false;
|
||||
|
||||
if (pendingCommit != null) {
|
||||
try {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "commit: pendingCommit != null");
|
||||
}
|
||||
pendingCommit.finishCommit(directory);
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "commit: wrote segments file \"" + pendingCommit.getSegmentsFileName() + "\"");
|
||||
success = true;
|
||||
// we committed, if anything goes wrong after this: we are screwed
|
||||
try {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "commit: wrote segments file \"" + pendingCommit.getSegmentsFileName() + "\"");
|
||||
}
|
||||
segmentInfos.updateGeneration(pendingCommit);
|
||||
lastCommitChangeCount = pendingCommitChangeCount;
|
||||
rollbackSegments = pendingCommit.createBackupSegmentInfos();
|
||||
// NOTE: don't use this.checkpoint() here, because
|
||||
// we do not want to increment changeCount:
|
||||
deleter.checkpoint(pendingCommit, true);
|
||||
} catch (Throwable tragedy) {
|
||||
tragicEvent(tragedy, "finishCommit");
|
||||
}
|
||||
segmentInfos.updateGeneration(pendingCommit);
|
||||
lastCommitChangeCount = pendingCommitChangeCount;
|
||||
rollbackSegments = pendingCommit.createBackupSegmentInfos();
|
||||
// NOTE: don't use this.checkpoint() here, because
|
||||
// we do not want to increment changeCount:
|
||||
deleter.checkpoint(pendingCommit, true);
|
||||
} finally {
|
||||
// Matches the incRef done in prepareCommit:
|
||||
try {
|
||||
deleter.decRef(filesToCommit);
|
||||
if (success == false || tragedy == null) {
|
||||
try {
|
||||
deleter.decRef(filesToCommit);
|
||||
} catch (Throwable t) {
|
||||
// if the commit succeeded, we are in screwed state
|
||||
// otherwise, throw our original exception
|
||||
if (success) {
|
||||
tragicEvent(tragedy, "finishCommit");
|
||||
}
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
filesToCommit = null;
|
||||
pendingCommit = null;
|
||||
|
@ -3045,8 +2981,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
|
||||
private boolean doFlush(boolean applyAllDeletes) throws IOException {
|
||||
if (hitOOM) {
|
||||
throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot flush");
|
||||
if (tragedy != null) {
|
||||
throw new IllegalStateException("this writer hit an unrecoverable error; cannot flush", tragedy);
|
||||
}
|
||||
|
||||
doBeforeFlush();
|
||||
|
@ -3081,7 +3017,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
return anySegmentFlushed;
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "doFlush");
|
||||
tragicEvent(oom, "doFlush");
|
||||
// never hit
|
||||
return false;
|
||||
} finally {
|
||||
|
@ -3407,8 +3343,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
|
||||
assert testPoint("startCommitMerge");
|
||||
|
||||
if (hitOOM) {
|
||||
throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot complete merge");
|
||||
if (tragedy != null) {
|
||||
throw new IllegalStateException("this writer hit an unrecoverable error; cannot complete merge", tragedy);
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
|
@ -3625,7 +3561,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "merge");
|
||||
tragicEvent(oom, "merge");
|
||||
}
|
||||
if (merge.info != null && !merge.isAborted()) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
|
@ -3754,8 +3690,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
assert merge.registerDone;
|
||||
assert merge.maxNumSegments == -1 || merge.maxNumSegments > 0;
|
||||
|
||||
if (hitOOM) {
|
||||
throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot merge");
|
||||
if (tragedy != null) {
|
||||
throw new IllegalStateException("this writer hit an unrecoverable error; cannot merge", tragedy);
|
||||
}
|
||||
|
||||
if (merge.info != null) {
|
||||
|
@ -4285,8 +4221,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
assert testPoint("startStartCommit");
|
||||
assert pendingCommit == null;
|
||||
|
||||
if (hitOOM) {
|
||||
throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot commit");
|
||||
if (tragedy != null) {
|
||||
throw new IllegalStateException("this writer hit an unrecoverable error; cannot commit", tragedy);
|
||||
}
|
||||
|
||||
try {
|
||||
|
@ -4384,7 +4320,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "startCommit");
|
||||
tragicEvent(oom, "startCommit");
|
||||
}
|
||||
assert testPoint("finishStartCommit");
|
||||
}
|
||||
|
@ -4436,12 +4372,26 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
public abstract void warm(AtomicReader reader) throws IOException;
|
||||
}
|
||||
|
||||
private void handleOOM(OutOfMemoryError oom, String location) {
|
||||
private void tragicEvent(Throwable tragedy, String location) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit OutOfMemoryError inside " + location);
|
||||
infoStream.message("IW", "hit " + tragedy.getClass().getSimpleName() + " inside " + location);
|
||||
}
|
||||
hitOOM = true;
|
||||
throw oom;
|
||||
// its possible you could have a really bad day
|
||||
if (this.tragedy == null) {
|
||||
this.tragedy = tragedy;
|
||||
}
|
||||
// if we are already closed (e.g. called by rollback), this will be a no-op.
|
||||
synchronized(commitLock) {
|
||||
if (closing == false) {
|
||||
try {
|
||||
rollback();
|
||||
} catch (Throwable ignored) {
|
||||
// it would be confusing to addSuppressed here, its unrelated to the disaster,
|
||||
// and its possible our internal state is amiss anyway.
|
||||
}
|
||||
}
|
||||
}
|
||||
IOUtils.reThrowUnchecked(tragedy);
|
||||
}
|
||||
|
||||
// Used only by assert for testing. Current points:
|
||||
|
|
|
@ -25,4 +25,8 @@ public class AlreadyClosedException extends IllegalStateException {
|
|||
public AlreadyClosedException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public AlreadyClosedException(String message, Throwable cause) {
|
||||
super(message, cause);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -341,7 +341,7 @@ public final class IOUtils {
|
|||
}
|
||||
|
||||
/**
|
||||
* Simple utilty method that takes a previously caught
|
||||
* Simple utility method that takes a previously caught
|
||||
* {@code Throwable} and rethrows either {@code
|
||||
* IOException} or an unchecked exception. If the
|
||||
* argument is null then this method does nothing.
|
||||
|
@ -356,7 +356,7 @@ public final class IOUtils {
|
|||
}
|
||||
|
||||
/**
|
||||
* Simple utilty method that takes a previously caught
|
||||
* Simple utility method that takes a previously caught
|
||||
* {@code Throwable} and rethrows it as an unchecked exception.
|
||||
* If the argument is null then this method does nothing.
|
||||
*/
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.lucene.codecs.Codec;
|
|||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -479,6 +480,8 @@ public class TestIndexFileDeleter extends LuceneTestCase {
|
|||
} catch (RuntimeException re) {
|
||||
if (re.getMessage().equals("fake fail")) {
|
||||
// ok
|
||||
} else if (re instanceof AlreadyClosedException && re.getCause() != null && "fake fail".equals(re.getCause().getMessage())) {
|
||||
break; // our test got unlucky, triggered our strange exception after successful finishCommit, caused a disaster!
|
||||
} else {
|
||||
throw re;
|
||||
}
|
||||
|
|
|
@ -37,6 +37,7 @@ import org.apache.lucene.document.SortedNumericDocValuesField;
|
|||
import org.apache.lucene.document.SortedSetDocValuesField;
|
||||
import org.apache.lucene.document.StoredField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||
import org.apache.lucene.store.MockDirectoryWrapper.Failure;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -136,17 +137,9 @@ 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 e) {
|
||||
if (e.getMessage() != null && e.getMessage().startsWith("Fake OutOfMemoryError")) {
|
||||
exceptionStream.println("\nTEST: got expected fake exc:" + e.getMessage());
|
||||
e.printStackTrace(exceptionStream);
|
||||
try {
|
||||
iw.rollback();
|
||||
} catch (Throwable t) {}
|
||||
continue STARTOVER;
|
||||
} else {
|
||||
Rethrow.rethrow(e);
|
||||
}
|
||||
} catch (OutOfMemoryError | AlreadyClosedException disaster) {
|
||||
getOOM(disaster, iw, exceptionStream);
|
||||
continue STARTOVER;
|
||||
}
|
||||
} else {
|
||||
// block docs
|
||||
|
@ -163,16 +156,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 e) {
|
||||
if (e.getMessage() != null && e.getMessage().startsWith("Fake OutOfMemoryError")) {
|
||||
exceptionStream.println("\nTEST: got expected fake exc:" + e.getMessage());
|
||||
e.printStackTrace(exceptionStream);
|
||||
} else {
|
||||
Rethrow.rethrow(e);
|
||||
}
|
||||
try {
|
||||
iw.rollback();
|
||||
} catch (Throwable t) {}
|
||||
} catch (OutOfMemoryError | AlreadyClosedException disaster) {
|
||||
getOOM(disaster, iw, exceptionStream);
|
||||
continue STARTOVER;
|
||||
}
|
||||
}
|
||||
|
@ -194,16 +179,8 @@ public class TestIndexWriterOutOfMemory extends LuceneTestCase {
|
|||
if (DirectoryReader.indexExists(dir)) {
|
||||
TestUtil.checkIndex(dir);
|
||||
}
|
||||
} catch (OutOfMemoryError e) {
|
||||
if (e.getMessage() != null && e.getMessage().startsWith("Fake OutOfMemoryError")) {
|
||||
exceptionStream.println("\nTEST: got expected fake exc:" + e.getMessage());
|
||||
e.printStackTrace(exceptionStream);
|
||||
} else {
|
||||
Rethrow.rethrow(e);
|
||||
}
|
||||
try {
|
||||
iw.rollback();
|
||||
} catch (Throwable t) {}
|
||||
} catch (OutOfMemoryError | AlreadyClosedException disaster) {
|
||||
getOOM(disaster, iw, exceptionStream);
|
||||
continue STARTOVER;
|
||||
}
|
||||
}
|
||||
|
@ -211,17 +188,9 @@ public class TestIndexWriterOutOfMemory extends LuceneTestCase {
|
|||
|
||||
try {
|
||||
iw.close();
|
||||
} catch (OutOfMemoryError e) {
|
||||
if (e.getMessage() != null && e.getMessage().startsWith("Fake OutOfMemoryError")) {
|
||||
exceptionStream.println("\nTEST: got expected fake exc:" + e.getMessage());
|
||||
e.printStackTrace(exceptionStream);
|
||||
try {
|
||||
iw.rollback();
|
||||
} catch (Throwable t) {}
|
||||
continue STARTOVER;
|
||||
} else {
|
||||
Rethrow.rethrow(e);
|
||||
}
|
||||
} catch (OutOfMemoryError | AlreadyClosedException disaster) {
|
||||
getOOM(disaster, iw, exceptionStream);
|
||||
continue STARTOVER;
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
System.out.println("Unexpected exception: dumping fake-exception-log:...");
|
||||
|
@ -238,6 +207,27 @@ public class TestIndexWriterOutOfMemory extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
private OutOfMemoryError getOOM(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")) {
|
||||
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
|
||||
// assertTrue("hit OOM but writer is still open, WTF: ", writer.isClosed());
|
||||
try {
|
||||
writer.rollback();
|
||||
} catch (Throwable t) {}
|
||||
return (OutOfMemoryError) e;
|
||||
} else {
|
||||
Rethrow.rethrow(disaster);
|
||||
return null; // dead
|
||||
}
|
||||
}
|
||||
|
||||
public void testBasics() throws Exception {
|
||||
final Random r = new Random(random().nextLong());
|
||||
doTest(new Failure() {
|
||||
|
@ -258,7 +248,6 @@ public class TestIndexWriterOutOfMemory extends LuceneTestCase {
|
|||
});
|
||||
}
|
||||
|
||||
@Ignore("LUCENE-5958: not yet")
|
||||
public void testCheckpoint() throws Exception {
|
||||
final Random r = new Random(random().nextLong());
|
||||
doTest(new Failure() {
|
||||
|
|
Loading…
Reference in New Issue