mirror of https://github.com/apache/lucene.git
LUCENE-6579: exceptions during merging are now tragic
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1690886 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
33d18a0c59
commit
926397824e
|
@ -301,6 +301,10 @@ Changes in Runtime Behavior
|
||||||
"used" when the first Scorer is pulled instead of when a Scorer is pulled on
|
"used" when the first Scorer is pulled instead of when a Scorer is pulled on
|
||||||
the first segment on an index. (Terry Smith, Adrien Grand)
|
the first segment on an index. (Terry Smith, Adrien Grand)
|
||||||
|
|
||||||
|
* LUCENE-6579: IndexWriter now sacrifices (closes) itself to protect the index
|
||||||
|
when an unexpected, tragic exception strikes while merging. (Robert
|
||||||
|
Muir, Mike McCandless)
|
||||||
|
|
||||||
Optimizations
|
Optimizations
|
||||||
|
|
||||||
* LUCENE-6548: Some optimizations for BlockTree's intersect with very
|
* LUCENE-6548: Some optimizations for BlockTree's intersect with very
|
||||||
|
|
|
@ -414,7 +414,9 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
|
||||||
MergeThread toSync = null;
|
MergeThread toSync = null;
|
||||||
synchronized (this) {
|
synchronized (this) {
|
||||||
for (MergeThread t : mergeThreads) {
|
for (MergeThread t : mergeThreads) {
|
||||||
if (t.isAlive()) {
|
// In case a merge thread is calling us, don't try to sync on
|
||||||
|
// itself, since that will never finish!
|
||||||
|
if (t.isAlive() && t != Thread.currentThread()) {
|
||||||
toSync = t;
|
toSync = t;
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
@ -663,18 +665,6 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
|
||||||
/** Called when an exception is hit in a background merge
|
/** Called when an exception is hit in a background merge
|
||||||
* thread */
|
* thread */
|
||||||
protected void handleMergeException(Directory dir, Throwable exc) {
|
protected void handleMergeException(Directory dir, Throwable exc) {
|
||||||
try {
|
|
||||||
// When an exception is hit during merge, IndexWriter
|
|
||||||
// removes any partial files and then allows another
|
|
||||||
// merge to run. If whatever caused the error is not
|
|
||||||
// transient then the exception will keep happening,
|
|
||||||
// so, we sleep here to avoid saturating CPU in such
|
|
||||||
// cases:
|
|
||||||
Thread.sleep(250);
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
throw new ThreadInterruptedException(ie);
|
|
||||||
}
|
|
||||||
|
|
||||||
throw new MergePolicy.MergeException(exc, dir);
|
throw new MergePolicy.MergeException(exc, dir);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -1055,7 +1055,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
}
|
}
|
||||||
// Ensure that only one thread actually gets to do the
|
// Ensure that only one thread actually gets to do the
|
||||||
// closing
|
// closing
|
||||||
if (shouldClose()) {
|
if (shouldClose(true)) {
|
||||||
boolean success = false;
|
boolean success = false;
|
||||||
try {
|
try {
|
||||||
if (infoStream.isEnabled("IW")) {
|
if (infoStream.isEnabled("IW")) {
|
||||||
|
@ -1113,14 +1113,17 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Returns true if this thread should attempt to close, or
|
// Returns true if this thread should attempt to close, or
|
||||||
// false if IndexWriter is now closed; else, waits until
|
// false if IndexWriter is now closed; else,
|
||||||
// another thread finishes closing
|
// waits until another thread finishes closing
|
||||||
synchronized private boolean shouldClose() {
|
synchronized private boolean shouldClose(boolean waitForClose) {
|
||||||
while(true) {
|
while (true) {
|
||||||
if (!closed) {
|
if (closed == false) {
|
||||||
if (!closing) {
|
if (closing == false) {
|
||||||
|
// We get to close
|
||||||
closing = true;
|
closing = true;
|
||||||
return true;
|
return true;
|
||||||
|
} else if (waitForClose == false) {
|
||||||
|
return false;
|
||||||
} else {
|
} else {
|
||||||
// Another thread is presently trying to close;
|
// Another thread is presently trying to close;
|
||||||
// wait until it finishes one way (closes
|
// wait until it finishes one way (closes
|
||||||
|
@ -2012,7 +2015,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
// Ensure that only one thread actually gets to do the
|
// Ensure that only one thread actually gets to do the
|
||||||
// closing, and make sure no commit is also in progress:
|
// closing, and make sure no commit is also in progress:
|
||||||
synchronized(commitLock) {
|
synchronized(commitLock) {
|
||||||
if (shouldClose()) {
|
if (shouldClose(true)) {
|
||||||
rollbackInternal();
|
rollbackInternal();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -2027,10 +2030,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
synchronized(this) {
|
abortMerges();
|
||||||
abortMerges();
|
|
||||||
stopMerges = true;
|
|
||||||
}
|
|
||||||
|
|
||||||
rateLimiters.close();
|
rateLimiters.close();
|
||||||
|
|
||||||
|
@ -2039,7 +2039,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Must pre-close in case it increments changeCount so that we can then
|
// Must pre-close in case it increments changeCount so that we can then
|
||||||
// set it to false before calling closeInternal
|
// set it to false before calling rollbackInternal
|
||||||
mergeScheduler.close();
|
mergeScheduler.close();
|
||||||
|
|
||||||
bufferedUpdatesStream.clear();
|
bufferedUpdatesStream.clear();
|
||||||
|
@ -2061,11 +2061,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
readerPool.dropAll(false);
|
readerPool.dropAll(false);
|
||||||
|
|
||||||
// Keep the same segmentInfos instance but replace all
|
// Keep the same segmentInfos instance but replace all
|
||||||
// of its SegmentInfo instances. This is so the next
|
// of its SegmentInfo instances so IFD below will remove
|
||||||
// attempt to commit using this instance of IndexWriter
|
// any segments we flushed since the last commit:
|
||||||
// will always write to a new generation ("write
|
|
||||||
// once").
|
|
||||||
segmentInfos.rollbackSegmentInfos(rollbackSegments);
|
segmentInfos.rollbackSegmentInfos(rollbackSegments);
|
||||||
|
|
||||||
if (infoStream.isEnabled("IW") ) {
|
if (infoStream.isEnabled("IW") ) {
|
||||||
infoStream.message("IW", "rollback: infos=" + segString(segmentInfos));
|
infoStream.message("IW", "rollback: infos=" + segString(segmentInfos));
|
||||||
}
|
}
|
||||||
|
@ -2073,13 +2072,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
testPoint("rollback before checkpoint");
|
testPoint("rollback before checkpoint");
|
||||||
|
|
||||||
// Ask deleter to locate unreferenced files & remove
|
// Ask deleter to locate unreferenced files & remove
|
||||||
// them:
|
// them ... only when we are not experiencing a tragedy, else
|
||||||
deleter.checkpoint(segmentInfos, false);
|
// these methods throw ACE:
|
||||||
deleter.refresh();
|
if (tragedy == null) {
|
||||||
|
deleter.checkpoint(segmentInfos, false);
|
||||||
|
deleter.refresh();
|
||||||
|
deleter.close();
|
||||||
|
}
|
||||||
|
|
||||||
lastCommitChangeCount = changeCount.get();
|
lastCommitChangeCount = changeCount.get();
|
||||||
|
|
||||||
deleter.close();
|
|
||||||
|
|
||||||
// Must set closed while inside same sync block where we call deleter.refresh, else concurrent threads may try to sneak a flush in,
|
// Must set closed while inside same sync block where we call deleter.refresh, else concurrent threads may try to sneak a flush in,
|
||||||
// after we leave this sync block and before we enter the sync block in the finally clause below that sets closed:
|
// after we leave this sync block and before we enter the sync block in the finally clause below that sets closed:
|
||||||
|
@ -2093,14 +2094,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
} catch (OutOfMemoryError oom) {
|
} catch (OutOfMemoryError oom) {
|
||||||
tragicEvent(oom, "rollbackInternal");
|
tragicEvent(oom, "rollbackInternal");
|
||||||
} finally {
|
} finally {
|
||||||
if (!success) {
|
if (success == false) {
|
||||||
// Must not hold IW's lock while closing
|
// Must not hold IW's lock while closing
|
||||||
// mergeScheduler: this can lead to deadlock,
|
// mergeScheduler: this can lead to deadlock,
|
||||||
// e.g. TestIW.testThreadInterruptDeadlock
|
// e.g. TestIW.testThreadInterruptDeadlock
|
||||||
IOUtils.closeWhileHandlingException(mergeScheduler);
|
IOUtils.closeWhileHandlingException(mergeScheduler);
|
||||||
}
|
}
|
||||||
synchronized(this) {
|
synchronized(this) {
|
||||||
if (!success) {
|
if (success == false) {
|
||||||
// we tried to be nice about it: do the minimum
|
// we tried to be nice about it: do the minimum
|
||||||
|
|
||||||
// don't leak a segments_N file if there is a pending commit
|
// don't leak a segments_N file if there is a pending commit
|
||||||
|
@ -2119,6 +2120,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
}
|
}
|
||||||
closed = true;
|
closed = true;
|
||||||
closing = false;
|
closing = false;
|
||||||
|
|
||||||
|
// So any "concurrently closing" threads wake up and see that the close has now completed:
|
||||||
|
notifyAll();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -2176,6 +2180,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
try {
|
try {
|
||||||
// Abort any running merges
|
// Abort any running merges
|
||||||
abortMerges();
|
abortMerges();
|
||||||
|
// Let merges run again
|
||||||
|
stopMerges = false;
|
||||||
// Remove all segments
|
// Remove all segments
|
||||||
pendingNumDocs.addAndGet(-segmentInfos.totalMaxDoc());
|
pendingNumDocs.addAndGet(-segmentInfos.totalMaxDoc());
|
||||||
segmentInfos.clear();
|
segmentInfos.clear();
|
||||||
|
@ -2214,6 +2220,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
* method: when you abort a long-running merge, you lose
|
* method: when you abort a long-running merge, you lose
|
||||||
* a lot of work that must later be redone. */
|
* a lot of work that must later be redone. */
|
||||||
private synchronized void abortMerges() {
|
private synchronized void abortMerges() {
|
||||||
|
|
||||||
stopMerges = true;
|
stopMerges = true;
|
||||||
|
|
||||||
// Abort all pending & running merges:
|
// Abort all pending & running merges:
|
||||||
|
@ -2233,21 +2240,19 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
merge.rateLimiter.setAbort();
|
merge.rateLimiter.setAbort();
|
||||||
}
|
}
|
||||||
|
|
||||||
// These merges periodically check whether they have
|
// We wait here to make all merges stop. It should not
|
||||||
// been aborted, and stop if so. We wait here to make
|
// take very long because they periodically check if
|
||||||
// sure they all stop. It should not take very long
|
|
||||||
// because the merge threads periodically check if
|
|
||||||
// they are aborted.
|
// they are aborted.
|
||||||
while(runningMerges.size() > 0) {
|
while (runningMerges.size() != 0) {
|
||||||
|
|
||||||
if (infoStream.isEnabled("IW")) {
|
if (infoStream.isEnabled("IW")) {
|
||||||
infoStream.message("IW", "now wait for " + runningMerges.size() + " running merge/s to abort");
|
infoStream.message("IW", "now wait for " + runningMerges.size() + " running merge/s to abort");
|
||||||
}
|
}
|
||||||
|
|
||||||
doWait();
|
doWait();
|
||||||
}
|
}
|
||||||
|
|
||||||
stopMerges = false;
|
|
||||||
notifyAll();
|
notifyAll();
|
||||||
|
|
||||||
assert 0 == mergingSegments.size();
|
assert 0 == mergingSegments.size();
|
||||||
|
|
||||||
if (infoStream.isEnabled("IW")) {
|
if (infoStream.isEnabled("IW")) {
|
||||||
|
@ -2274,7 +2279,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
infoStream.message("IW", "waitForMerges");
|
infoStream.message("IW", "waitForMerges");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
while (pendingMerges.size() > 0 || runningMerges.size() > 0) {
|
while (pendingMerges.size() > 0 || runningMerges.size() > 0) {
|
||||||
doWait();
|
doWait();
|
||||||
}
|
}
|
||||||
|
@ -2975,6 +2979,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
|
|
||||||
try {
|
try {
|
||||||
synchronized(this) {
|
synchronized(this) {
|
||||||
|
ensureOpen(false);
|
||||||
|
|
||||||
|
if (tragedy != null) {
|
||||||
|
throw new IllegalStateException("this writer hit an unrecoverable error; cannot complete commit", tragedy);
|
||||||
|
}
|
||||||
|
|
||||||
if (pendingCommit != null) {
|
if (pendingCommit != null) {
|
||||||
try {
|
try {
|
||||||
|
|
||||||
|
@ -3057,6 +3067,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
* deletes or docs were flushed) if necessary
|
* deletes or docs were flushed) if necessary
|
||||||
* @param applyAllDeletes whether pending deletes should also
|
* @param applyAllDeletes whether pending deletes should also
|
||||||
*/
|
*/
|
||||||
|
// why protected
|
||||||
protected final void flush(boolean triggerMerge, boolean applyAllDeletes) throws IOException {
|
protected final void flush(boolean triggerMerge, boolean applyAllDeletes) throws IOException {
|
||||||
|
|
||||||
// NOTE: this method cannot be sync'd because
|
// NOTE: this method cannot be sync'd because
|
||||||
|
@ -3596,7 +3607,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
|
|
||||||
if (t instanceof MergePolicy.MergeAbortedException) {
|
if (t instanceof MergePolicy.MergeAbortedException) {
|
||||||
// We can ignore this exception (it happens when
|
// We can ignore this exception (it happens when
|
||||||
// close(false) or rollback is called), unless the
|
// deleteAll or rollback is called), unless the
|
||||||
// merge involves segments from external directories,
|
// merge involves segments from external directories,
|
||||||
// in which case we must throw it so, for example, the
|
// in which case we must throw it so, for example, the
|
||||||
// rollbackTransaction code in addIndexes* is
|
// rollbackTransaction code in addIndexes* is
|
||||||
|
@ -3654,19 +3665,20 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
if (merge.info != null && !segmentInfos.contains(merge.info)) {
|
if (merge.info != null && !segmentInfos.contains(merge.info)) {
|
||||||
deleter.refresh(merge.info.info.name);
|
deleter.refresh(merge.info.info.name);
|
||||||
}
|
}
|
||||||
}
|
} else if (merge.rateLimiter.getAbort() == false && (merge.maxNumSegments != -1 || (!closed && !closing))) {
|
||||||
|
// This merge (and, generally, any change to the
|
||||||
// This merge (and, generally, any change to the
|
// segments) may now enable new merges, so we call
|
||||||
// segments) may now enable new merges, so we call
|
// merge policy & update pending merges.
|
||||||
// merge policy & update pending merges.
|
|
||||||
if (success && merge.rateLimiter.getAbort() == false && (merge.maxNumSegments != -1 || (!closed && !closing))) {
|
|
||||||
updatePendingMerges(mergePolicy, MergeTrigger.MERGE_FINISHED, merge.maxNumSegments);
|
updatePendingMerges(mergePolicy, MergeTrigger.MERGE_FINISHED, merge.maxNumSegments);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} catch (OutOfMemoryError oom) {
|
} catch (Throwable t) {
|
||||||
tragicEvent(oom, "merge");
|
// Important that tragicEvent is called after mergeFinish, else we hang
|
||||||
|
// waiting for our merge thread to be removed from runningMerges:
|
||||||
|
tragicEvent(t, "merge");
|
||||||
}
|
}
|
||||||
|
|
||||||
if (merge.info != null && merge.rateLimiter.getAbort() == false) {
|
if (merge.info != null && merge.rateLimiter.getAbort() == false) {
|
||||||
if (infoStream.isEnabled("IW")) {
|
if (infoStream.isEnabled("IW")) {
|
||||||
infoStream.message("IW", "merge time " + (System.currentTimeMillis()-t0) + " msec for " + merge.info.info.maxDoc() + " docs");
|
infoStream.message("IW", "merge time " + (System.currentTimeMillis()-t0) + " msec for " + merge.info.info.maxDoc() + " docs");
|
||||||
|
@ -4260,7 +4272,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
* debugging.
|
* debugging.
|
||||||
*
|
*
|
||||||
* @lucene.internal */
|
* @lucene.internal */
|
||||||
public synchronized String segString() {
|
synchronized String segString() {
|
||||||
return segString(segmentInfos);
|
return segString(segmentInfos);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -4268,7 +4280,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
* segments, for debugging.
|
* segments, for debugging.
|
||||||
*
|
*
|
||||||
* @lucene.internal */
|
* @lucene.internal */
|
||||||
public synchronized String segString(Iterable<SegmentCommitInfo> infos) {
|
synchronized String segString(Iterable<SegmentCommitInfo> infos) {
|
||||||
final StringBuilder buffer = new StringBuilder();
|
final StringBuilder buffer = new StringBuilder();
|
||||||
for(final SegmentCommitInfo info : infos) {
|
for(final SegmentCommitInfo info : infos) {
|
||||||
if (buffer.length() > 0) {
|
if (buffer.length() > 0) {
|
||||||
|
@ -4283,7 +4295,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
* segment, for debugging.
|
* segment, for debugging.
|
||||||
*
|
*
|
||||||
* @lucene.internal */
|
* @lucene.internal */
|
||||||
public synchronized String segString(SegmentCommitInfo info) {
|
synchronized String segString(SegmentCommitInfo info) {
|
||||||
return info.toString(numDeletedDocs(info) - info.getDelCount());
|
return info.toString(numDeletedDocs(info) - info.getDelCount());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -4513,10 +4525,16 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
}
|
}
|
||||||
|
|
||||||
void tragicEvent(Throwable tragedy, String location) throws IOException {
|
void tragicEvent(Throwable tragedy, String location) throws IOException {
|
||||||
|
|
||||||
// unbox our internal AbortingException
|
// unbox our internal AbortingException
|
||||||
if (tragedy instanceof AbortingException) {
|
if (tragedy instanceof AbortingException) {
|
||||||
tragedy = tragedy.getCause();
|
tragedy = tragedy.getCause();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// This is not supposed to be tragic: IW is supposed to catch this and
|
||||||
|
// ignore, because it means we asked the merge to abort:
|
||||||
|
assert tragedy instanceof MergePolicy.MergeAbortedException == false;
|
||||||
|
|
||||||
// We cannot hold IW's lock here else it can lead to deadlock:
|
// We cannot hold IW's lock here else it can lead to deadlock:
|
||||||
assert Thread.holdsLock(this) == false;
|
assert Thread.holdsLock(this) == false;
|
||||||
|
|
||||||
|
@ -4528,22 +4546,18 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
}
|
}
|
||||||
|
|
||||||
synchronized (this) {
|
synchronized (this) {
|
||||||
// it's possible you could have a really bad day
|
// It's possible you could have a really bad day
|
||||||
if (this.tragedy == null) {
|
if (this.tragedy != null) {
|
||||||
this.tragedy = tragedy;
|
// Another thread is already dealing / has dealt with the tragedy:
|
||||||
|
IOUtils.reThrow(tragedy);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
this.tragedy = tragedy;
|
||||||
}
|
}
|
||||||
|
|
||||||
// if we are already closed (e.g. called by rollback), this will be a no-op.
|
// if we are already closed (e.g. called by rollback), this will be a no-op.
|
||||||
synchronized(commitLock) {
|
if (shouldClose(false)) {
|
||||||
if (closing == false) {
|
rollbackInternal();
|
||||||
try {
|
|
||||||
rollback();
|
|
||||||
} catch (Throwable ignored) {
|
|
||||||
// it would be confusing to addSuppressed here, it's unrelated to the disaster,
|
|
||||||
// and it's possible our internal state is amiss anyway.
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
IOUtils.reThrow(tragedy);
|
IOUtils.reThrow(tragedy);
|
||||||
|
|
|
@ -79,8 +79,9 @@ public class TestMergeSchedulerExternal extends LuceneTestCase {
|
||||||
public void eval(MockDirectoryWrapper dir) throws IOException {
|
public void eval(MockDirectoryWrapper dir) throws IOException {
|
||||||
StackTraceElement[] trace = new Exception().getStackTrace();
|
StackTraceElement[] trace = new Exception().getStackTrace();
|
||||||
for (int i = 0; i < trace.length; i++) {
|
for (int i = 0; i < trace.length; i++) {
|
||||||
if ("doMerge".equals(trace[i].getMethodName()))
|
if ("doMerge".equals(trace[i].getMethodName())) {
|
||||||
throw new IOException("now failing during merge");
|
throw new IOException("now failing during merge");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -99,11 +100,16 @@ public class TestMergeSchedulerExternal extends LuceneTestCase {
|
||||||
.setMergePolicy(newLogMergePolicy()));
|
.setMergePolicy(newLogMergePolicy()));
|
||||||
LogMergePolicy logMP = (LogMergePolicy) writer.getConfig().getMergePolicy();
|
LogMergePolicy logMP = (LogMergePolicy) writer.getConfig().getMergePolicy();
|
||||||
logMP.setMergeFactor(10);
|
logMP.setMergeFactor(10);
|
||||||
for(int i=0;i<20;i++)
|
for(int i=0;i<20;i++) {
|
||||||
writer.addDocument(doc);
|
writer.addDocument(doc);
|
||||||
|
}
|
||||||
|
|
||||||
((MyMergeScheduler) writer.getConfig().getMergeScheduler()).sync();
|
try {
|
||||||
writer.close();
|
((MyMergeScheduler) writer.getConfig().getMergeScheduler()).sync();
|
||||||
|
} catch (IllegalStateException ise) {
|
||||||
|
// OK
|
||||||
|
}
|
||||||
|
writer.rollback();
|
||||||
|
|
||||||
assertTrue(mergeThreadCreated);
|
assertTrue(mergeThreadCreated);
|
||||||
assertTrue(mergeCalled);
|
assertTrue(mergeCalled);
|
||||||
|
|
|
@ -417,7 +417,7 @@ public class TestIndexFileDeleter extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
// LUCENE-5919
|
// LUCENE-5919
|
||||||
public void testExcInDecRef() throws Exception {
|
public void testExcInDecRef() throws Throwable {
|
||||||
MockDirectoryWrapper dir = newMockDirectory();
|
MockDirectoryWrapper dir = newMockDirectory();
|
||||||
|
|
||||||
// disable slow things: we don't rely upon sleeps here.
|
// disable slow things: we don't rely upon sleeps here.
|
||||||
|
@ -484,24 +484,11 @@ public class TestIndexFileDeleter extends LuceneTestCase {
|
||||||
doc.add(newTextField("field", "some text", Field.Store.NO));
|
doc.add(newTextField("field", "some text", Field.Store.NO));
|
||||||
w.addDocument(doc);
|
w.addDocument(doc);
|
||||||
}
|
}
|
||||||
} catch (IOException ioe) {
|
} catch (Throwable t) {
|
||||||
if (ioe.getMessage().contains("background merge hit exception")) {
|
if (t.toString().contains("fake fail") || t.getCause().toString().contains("fake fail")) {
|
||||||
Throwable cause = ioe.getCause();
|
|
||||||
if (cause != null && cause instanceof RuntimeException && ((RuntimeException) cause).getMessage().equals("fake fail")) {
|
|
||||||
// ok
|
|
||||||
} else {
|
|
||||||
throw ioe;
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
throw ioe;
|
|
||||||
}
|
|
||||||
} catch (RuntimeException re) {
|
|
||||||
if (re.getMessage().equals("fake fail")) {
|
|
||||||
// ok
|
// 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 {
|
} else {
|
||||||
throw re;
|
throw t;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -598,7 +598,12 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
docId += 12;
|
docId += 12;
|
||||||
}
|
}
|
||||||
modifier.close();
|
try {
|
||||||
|
modifier.close();
|
||||||
|
} catch (IllegalStateException ise) {
|
||||||
|
// ok
|
||||||
|
throw (IOException) ise.getCause();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
success = true;
|
success = true;
|
||||||
if (0 == x) {
|
if (0 == x) {
|
||||||
|
@ -846,6 +851,10 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||||
modifier.commit();
|
modifier.commit();
|
||||||
} catch (RuntimeException ioe) {
|
} catch (RuntimeException ioe) {
|
||||||
// expected
|
// expected
|
||||||
|
if (VERBOSE) {
|
||||||
|
System.out.println("TEST: hit exc:");
|
||||||
|
ioe.printStackTrace(System.out);
|
||||||
|
}
|
||||||
failed = true;
|
failed = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -854,14 +863,23 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||||
// The commit above failed, so we need to retry it (which will
|
// The commit above failed, so we need to retry it (which will
|
||||||
// succeed, because the failure is a one-shot)
|
// succeed, because the failure is a one-shot)
|
||||||
|
|
||||||
modifier.commit();
|
boolean writerClosed;
|
||||||
|
try {
|
||||||
|
modifier.commit();
|
||||||
|
writerClosed = false;
|
||||||
|
} catch (IllegalStateException ise) {
|
||||||
|
// The above exc struck during merge, and closed the writer
|
||||||
|
writerClosed = true;
|
||||||
|
}
|
||||||
|
|
||||||
hitCount = getHitCount(dir, term);
|
if (writerClosed == false) {
|
||||||
|
hitCount = getHitCount(dir, term);
|
||||||
|
|
||||||
// Make sure the delete was successfully flushed:
|
// Make sure the delete was successfully flushed:
|
||||||
assertEquals(0, hitCount);
|
assertEquals(0, hitCount);
|
||||||
|
|
||||||
modifier.close();
|
modifier.close();
|
||||||
|
}
|
||||||
dir.close();
|
dir.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1361,4 +1379,38 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
||||||
w.close();
|
w.close();
|
||||||
dir.close();
|
dir.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Make sure merges still kick off after IW.deleteAll!
|
||||||
|
public void testMergingAfterDeleteAll() throws Exception {
|
||||||
|
Directory dir = newDirectory();
|
||||||
|
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||||
|
iwc.setMaxBufferedDocs(2);
|
||||||
|
LogDocMergePolicy mp = new LogDocMergePolicy();
|
||||||
|
mp.setMinMergeDocs(1);
|
||||||
|
iwc.setMergePolicy(mp);
|
||||||
|
iwc.setMergeScheduler(new SerialMergeScheduler());
|
||||||
|
IndexWriter w = new IndexWriter(dir, iwc);
|
||||||
|
for(int i=0;i<10;i++) {
|
||||||
|
Document doc = new Document();
|
||||||
|
doc.add(newStringField("id", ""+i, Field.Store.NO));
|
||||||
|
w.addDocument(doc);
|
||||||
|
}
|
||||||
|
w.commit();
|
||||||
|
w.deleteAll();
|
||||||
|
|
||||||
|
for(int i=0;i<100;i++) {
|
||||||
|
Document doc = new Document();
|
||||||
|
doc.add(newStringField("id", ""+i, Field.Store.NO));
|
||||||
|
w.addDocument(doc);
|
||||||
|
}
|
||||||
|
|
||||||
|
w.forceMerge(1);
|
||||||
|
|
||||||
|
DirectoryReader r = DirectoryReader.open(w, true);
|
||||||
|
assertEquals(1, r.leaves().size());
|
||||||
|
r.close();
|
||||||
|
|
||||||
|
w.close();
|
||||||
|
dir.close();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -456,14 +456,19 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
|
||||||
testPoint.doFail = true;
|
testPoint.doFail = true;
|
||||||
Document doc = new Document();
|
Document doc = new Document();
|
||||||
doc.add(newTextField("field", "a field", Field.Store.YES));
|
doc.add(newTextField("field", "a field", Field.Store.YES));
|
||||||
for(int i=0;i<10;i++)
|
for(int i=0;i<10;i++) {
|
||||||
try {
|
try {
|
||||||
w.addDocument(doc);
|
w.addDocument(doc);
|
||||||
} catch (RuntimeException re) {
|
} catch (RuntimeException re) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
((ConcurrentMergeScheduler) w.getConfig().getMergeScheduler()).sync();
|
try {
|
||||||
|
((ConcurrentMergeScheduler) w.getConfig().getMergeScheduler()).sync();
|
||||||
|
} catch (IllegalStateException ise) {
|
||||||
|
// OK: merge exc causes tragedy
|
||||||
|
}
|
||||||
assertTrue(testPoint.failed);
|
assertTrue(testPoint.failed);
|
||||||
w.close();
|
w.close();
|
||||||
dir.close();
|
dir.close();
|
||||||
|
@ -974,14 +979,15 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
|
||||||
.setMergePolicy(newLogMergePolicy());
|
.setMergePolicy(newLogMergePolicy());
|
||||||
((LogMergePolicy) conf.getMergePolicy()).setMergeFactor(100);
|
((LogMergePolicy) conf.getMergePolicy()).setMergeFactor(100);
|
||||||
IndexWriter w = new IndexWriter(startDir, conf);
|
IndexWriter w = new IndexWriter(startDir, conf);
|
||||||
for(int i=0;i<27;i++)
|
for(int i=0;i<27;i++) {
|
||||||
addDoc(w);
|
addDoc(w);
|
||||||
|
}
|
||||||
w.close();
|
w.close();
|
||||||
|
|
||||||
int iter = TEST_NIGHTLY ? 200 : 10;
|
int iter = TEST_NIGHTLY ? 200 : 10;
|
||||||
for(int i=0;i<iter;i++) {
|
for(int i=0;i<iter;i++) {
|
||||||
if (VERBOSE) {
|
if (VERBOSE) {
|
||||||
System.out.println("TEST: iter " + i);
|
System.out.println("\nTEST: iter " + i);
|
||||||
}
|
}
|
||||||
MockDirectoryWrapper dir = new MockDirectoryWrapper(random(), TestUtil.ramCopyOf(startDir));
|
MockDirectoryWrapper dir = new MockDirectoryWrapper(random(), TestUtil.ramCopyOf(startDir));
|
||||||
conf = newIndexWriterConfig(new MockAnalyzer(random()))
|
conf = newIndexWriterConfig(new MockAnalyzer(random()))
|
||||||
|
@ -991,12 +997,20 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
|
||||||
dir.setRandomIOExceptionRate(0.5);
|
dir.setRandomIOExceptionRate(0.5);
|
||||||
try {
|
try {
|
||||||
w.forceMerge(1);
|
w.forceMerge(1);
|
||||||
|
} catch (IllegalStateException ise) {
|
||||||
|
// expected
|
||||||
} catch (IOException ioe) {
|
} catch (IOException ioe) {
|
||||||
if (ioe.getCause() == null)
|
if (ioe.getCause() == null) {
|
||||||
fail("forceMerge threw IOException without root cause");
|
fail("forceMerge threw IOException without root cause");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
dir.setRandomIOExceptionRate(0);
|
dir.setRandomIOExceptionRate(0);
|
||||||
w.close();
|
//System.out.println("TEST: now close IW");
|
||||||
|
try {
|
||||||
|
w.close();
|
||||||
|
} catch (IllegalStateException ise) {
|
||||||
|
// ok
|
||||||
|
}
|
||||||
dir.close();
|
dir.close();
|
||||||
}
|
}
|
||||||
startDir.close();
|
startDir.close();
|
||||||
|
@ -2272,4 +2286,72 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
|
||||||
dir.close();
|
dir.close();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testMergeExceptionIsTragic() throws Exception {
|
||||||
|
MockDirectoryWrapper dir = newMockDirectory();
|
||||||
|
final AtomicBoolean didFail = new AtomicBoolean();
|
||||||
|
dir.failOn(new MockDirectoryWrapper.Failure() {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void eval(MockDirectoryWrapper dir) throws IOException {
|
||||||
|
if (random().nextInt(10) != 0) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
if (didFail.get()) {
|
||||||
|
// Already failed
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
StackTraceElement[] trace = Thread.currentThread().getStackTrace();
|
||||||
|
|
||||||
|
for (int i = 0; i < trace.length; i++) {
|
||||||
|
if ("merge".equals(trace[i].getMethodName())) {
|
||||||
|
if (VERBOSE) {
|
||||||
|
System.out.println("TEST: now fail; thread=" + Thread.currentThread().getName() + " exc:");
|
||||||
|
new Throwable().printStackTrace(System.out);
|
||||||
|
}
|
||||||
|
didFail.set(true);
|
||||||
|
throw new FakeIOException();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
IndexWriterConfig iwc = newIndexWriterConfig();
|
||||||
|
MergeScheduler ms = iwc.getMergeScheduler();
|
||||||
|
if (ms instanceof ConcurrentMergeScheduler) {
|
||||||
|
((ConcurrentMergeScheduler) ms).setSuppressExceptions();
|
||||||
|
}
|
||||||
|
IndexWriter w = new IndexWriter(dir, iwc);
|
||||||
|
|
||||||
|
while (true) {
|
||||||
|
try {
|
||||||
|
Document doc = new Document();
|
||||||
|
doc.add(newStringField("field", "string", Field.Store.NO));
|
||||||
|
w.addDocument(doc);
|
||||||
|
if (random().nextInt(10) == 7) {
|
||||||
|
// Flush new segment:
|
||||||
|
DirectoryReader.open(w, true).close();
|
||||||
|
}
|
||||||
|
} catch (AlreadyClosedException ace) {
|
||||||
|
// OK: e.g. CMS hit the exc in BG thread and closed the writer
|
||||||
|
break;
|
||||||
|
} catch (FakeIOException fioe) {
|
||||||
|
// OK: e.g. SMS hit the exception
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
assertNotNull(w.getTragicException());
|
||||||
|
assertFalse(w.isOpen());
|
||||||
|
assertTrue(didFail.get());
|
||||||
|
|
||||||
|
if (ms instanceof ConcurrentMergeScheduler) {
|
||||||
|
// Sneaky: CMS's merge thread will be concurrently rolling back IW due
|
||||||
|
// to the tragedy, with this main thread, so we have to wait here
|
||||||
|
// to ensure the rollback has finished, else MDW still sees open files:
|
||||||
|
((ConcurrentMergeScheduler) ms).sync();
|
||||||
|
}
|
||||||
|
|
||||||
|
dir.close();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -154,8 +154,7 @@ public class TestIndexWriterOnDiskFull extends LuceneTestCase {
|
||||||
either all or none of the incoming documents were in
|
either all or none of the incoming documents were in
|
||||||
fact added.
|
fact added.
|
||||||
*/
|
*/
|
||||||
public void testAddIndexOnDiskFull() throws IOException
|
public void testAddIndexOnDiskFull() throws IOException {
|
||||||
{
|
|
||||||
// MemoryCodec, since it uses FST, is not necessarily
|
// MemoryCodec, since it uses FST, is not necessarily
|
||||||
// "additive", ie if you add up N small FSTs, then merge
|
// "additive", ie if you add up N small FSTs, then merge
|
||||||
// them, the merged result can easily be larger than the
|
// them, the merged result can easily be larger than the
|
||||||
|
@ -256,21 +255,25 @@ public class TestIndexWriterOnDiskFull extends LuceneTestCase {
|
||||||
|
|
||||||
// Make a new dir that will enforce disk usage:
|
// Make a new dir that will enforce disk usage:
|
||||||
MockDirectoryWrapper dir = new MockDirectoryWrapper(random(), TestUtil.ramCopyOf(startDir));
|
MockDirectoryWrapper dir = new MockDirectoryWrapper(random(), TestUtil.ramCopyOf(startDir));
|
||||||
writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
|
dir.setPreventDoubleWrite(false);
|
||||||
.setOpenMode(OpenMode.APPEND)
|
IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()))
|
||||||
.setMergePolicy(newLogMergePolicy(false)));
|
.setOpenMode(OpenMode.APPEND)
|
||||||
IOException err = null;
|
.setMergePolicy(newLogMergePolicy(false));
|
||||||
|
writer = new IndexWriter(dir, iwc);
|
||||||
|
Exception err = null;
|
||||||
|
|
||||||
MergeScheduler ms = writer.getConfig().getMergeScheduler();
|
|
||||||
for(int x=0;x<2;x++) {
|
for(int x=0;x<2;x++) {
|
||||||
if (ms instanceof ConcurrentMergeScheduler)
|
MergeScheduler ms = writer.getConfig().getMergeScheduler();
|
||||||
|
if (ms instanceof ConcurrentMergeScheduler) {
|
||||||
// This test intentionally produces exceptions
|
// This test intentionally produces exceptions
|
||||||
// in the threads that CMS launches; we don't
|
// in the threads that CMS launches; we don't
|
||||||
// want to pollute test output with these.
|
// want to pollute test output with these.
|
||||||
if (0 == x)
|
if (0 == x) {
|
||||||
((ConcurrentMergeScheduler) ms).setSuppressExceptions();
|
((ConcurrentMergeScheduler) ms).setSuppressExceptions();
|
||||||
else
|
} else {
|
||||||
((ConcurrentMergeScheduler) ms).clearSuppressExceptions();
|
((ConcurrentMergeScheduler) ms).clearSuppressExceptions();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// Two loops: first time, limit disk space &
|
// Two loops: first time, limit disk space &
|
||||||
// throw random IOExceptions; second time, no
|
// throw random IOExceptions; second time, no
|
||||||
|
@ -294,18 +297,21 @@ public class TestIndexWriterOnDiskFull extends LuceneTestCase {
|
||||||
if (diskRatio >= 6.0) {
|
if (diskRatio >= 6.0) {
|
||||||
rate = 0.0;
|
rate = 0.0;
|
||||||
}
|
}
|
||||||
if (VERBOSE)
|
if (VERBOSE) {
|
||||||
testName = "disk full test " + methodName + " with disk full at " + diskFree + " bytes";
|
testName = "disk full test " + methodName + " with disk full at " + diskFree + " bytes";
|
||||||
|
}
|
||||||
} else {
|
} else {
|
||||||
dir.setRandomIOExceptionRateOnOpen(0.0);
|
dir.setRandomIOExceptionRateOnOpen(0.0);
|
||||||
thisDiskFree = 0;
|
thisDiskFree = 0;
|
||||||
rate = 0.0;
|
rate = 0.0;
|
||||||
if (VERBOSE)
|
if (VERBOSE) {
|
||||||
testName = "disk full test " + methodName + " with unlimited disk space";
|
testName = "disk full test " + methodName + " with unlimited disk space";
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (VERBOSE)
|
if (VERBOSE) {
|
||||||
System.out.println("\ncycle: " + testName);
|
System.out.println("\ncycle: " + testName);
|
||||||
|
}
|
||||||
|
|
||||||
dir.setTrackDiskUsage(true);
|
dir.setTrackDiskUsage(true);
|
||||||
dir.setMaxSizeInBytes(thisDiskFree);
|
dir.setMaxSizeInBytes(thisDiskFree);
|
||||||
|
@ -347,11 +353,11 @@ public class TestIndexWriterOnDiskFull extends LuceneTestCase {
|
||||||
done = true;
|
done = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
} catch (IOException e) {
|
} catch (IllegalStateException | IOException e) {
|
||||||
success = false;
|
success = false;
|
||||||
err = e;
|
err = e;
|
||||||
if (VERBOSE) {
|
if (VERBOSE) {
|
||||||
System.out.println(" hit IOException: " + e);
|
System.out.println(" hit Exception: " + e);
|
||||||
e.printStackTrace(System.out);
|
e.printStackTrace(System.out);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -361,9 +367,16 @@ public class TestIndexWriterOnDiskFull extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Make sure all threads from
|
if (x == 1) {
|
||||||
// ConcurrentMergeScheduler are done
|
// Make sure all threads from ConcurrentMergeScheduler are done
|
||||||
TestUtil.syncConcurrentMerges(writer);
|
TestUtil.syncConcurrentMerges(writer);
|
||||||
|
} else {
|
||||||
|
dir.setRandomIOExceptionRateOnOpen(0.0);
|
||||||
|
writer.rollback();
|
||||||
|
writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
|
||||||
|
.setOpenMode(OpenMode.APPEND)
|
||||||
|
.setMergePolicy(newLogMergePolicy(false)));
|
||||||
|
}
|
||||||
|
|
||||||
if (VERBOSE) {
|
if (VERBOSE) {
|
||||||
System.out.println(" now test readers");
|
System.out.println(" now test readers");
|
||||||
|
@ -447,11 +460,6 @@ public class TestIndexWriterOnDiskFull extends LuceneTestCase {
|
||||||
|
|
||||||
writer.close();
|
writer.close();
|
||||||
|
|
||||||
// Wait for all BG threads to finish else
|
|
||||||
// dir.close() will throw IOException because
|
|
||||||
// there are still open files
|
|
||||||
TestUtil.syncConcurrentMerges(ms);
|
|
||||||
|
|
||||||
dir.close();
|
dir.close();
|
||||||
|
|
||||||
// Try again with more free space:
|
// Try again with more free space:
|
||||||
|
@ -524,8 +532,12 @@ public class TestIndexWriterOnDiskFull extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
TestUtil.checkIndex(dir);
|
TestUtil.checkIndex(dir);
|
||||||
ftdm.clearDoFail();
|
ftdm.clearDoFail();
|
||||||
w.addDocument(doc);
|
try {
|
||||||
w.close();
|
w.addDocument(doc);
|
||||||
|
fail("writer was not closed by merge exception");
|
||||||
|
} catch (AlreadyClosedException ace) {
|
||||||
|
// expected
|
||||||
|
}
|
||||||
|
|
||||||
dir.close();
|
dir.close();
|
||||||
}
|
}
|
||||||
|
@ -556,20 +568,18 @@ public class TestIndexWriterOnDiskFull extends LuceneTestCase {
|
||||||
|
|
||||||
// TODO: these are also in TestIndexWriter... add a simple doc-writing method
|
// TODO: these are also in TestIndexWriter... add a simple doc-writing method
|
||||||
// like this to LuceneTestCase?
|
// like this to LuceneTestCase?
|
||||||
private void addDoc(IndexWriter writer) throws IOException
|
private void addDoc(IndexWriter writer) throws IOException {
|
||||||
{
|
Document doc = new Document();
|
||||||
Document doc = new Document();
|
doc.add(newTextField("content", "aaa", Field.Store.NO));
|
||||||
doc.add(newTextField("content", "aaa", Field.Store.NO));
|
doc.add(new NumericDocValuesField("numericdv", 1));
|
||||||
doc.add(new NumericDocValuesField("numericdv", 1));
|
writer.addDocument(doc);
|
||||||
writer.addDocument(doc);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void addDocWithIndex(IndexWriter writer, int index) throws IOException
|
private void addDocWithIndex(IndexWriter writer, int index) throws IOException {
|
||||||
{
|
Document doc = new Document();
|
||||||
Document doc = new Document();
|
doc.add(newTextField("content", "aaa " + index, Field.Store.NO));
|
||||||
doc.add(newTextField("content", "aaa " + index, Field.Store.NO));
|
doc.add(newTextField("id", "" + index, Field.Store.NO));
|
||||||
doc.add(newTextField("id", "" + index, Field.Store.NO));
|
doc.add(new NumericDocValuesField("numericdv", 1));
|
||||||
doc.add(new NumericDocValuesField("numericdv", 1));
|
writer.addDocument(doc);
|
||||||
writer.addDocument(doc);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -405,11 +405,11 @@ public class TestIDVersionPostingsFormat extends LuceneTestCase {
|
||||||
fail("didn't hit exception");
|
fail("didn't hit exception");
|
||||||
} catch (IllegalArgumentException iae) {
|
} catch (IllegalArgumentException iae) {
|
||||||
// expected: SMS will hit this
|
// expected: SMS will hit this
|
||||||
} catch (IOException ioe) {
|
} catch (IOException | IllegalStateException exc) {
|
||||||
// expected
|
// expected
|
||||||
assertTrue(ioe.getCause() instanceof IllegalArgumentException);
|
assertTrue(exc.getCause() instanceof IllegalArgumentException);
|
||||||
}
|
}
|
||||||
w.close();
|
w.rollback();
|
||||||
dir.close();
|
dir.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue