mirror of https://github.com/apache/lucene.git
LUCENE-1130: fix thread safety issues when hitting IOExceptions in DocumentsWriter
git-svn-id: https://svn.apache.org/repos/asf/lucene/java/trunk@611855 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
a43f312375
commit
f7740afe84
|
@ -251,7 +251,7 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
|
|||
|
||||
message(" merge thread: done");
|
||||
|
||||
} catch (IOException exc) {
|
||||
} catch (Throwable exc) {
|
||||
|
||||
if (merge != null) {
|
||||
merge.setException(exc);
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -1278,7 +1278,7 @@ public class IndexWriter {
|
|||
if (!success) {
|
||||
if (infoStream != null)
|
||||
message("hit exception closing doc store segment");
|
||||
docWriter.abort();
|
||||
docWriter.abort(null);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1999,7 +1999,7 @@ public class IndexWriter {
|
|||
segmentInfos.clear();
|
||||
segmentInfos.addAll(rollbackSegmentInfos);
|
||||
|
||||
docWriter.abort();
|
||||
docWriter.abort(null);
|
||||
|
||||
// Ask deleter to locate unreferenced files & remove
|
||||
// them:
|
||||
|
@ -2401,7 +2401,13 @@ public class IndexWriter {
|
|||
private synchronized final boolean doFlush(boolean flushDocStores) throws CorruptIndexException, IOException {
|
||||
|
||||
// Make sure no threads are actively adding a document
|
||||
docWriter.pauseAllThreads();
|
||||
|
||||
// Returns true if docWriter is currently aborting, in
|
||||
// which case we skip flushing this segment
|
||||
if (docWriter.pauseAllThreads()) {
|
||||
docWriter.resumeAllThreads();
|
||||
return false;
|
||||
}
|
||||
|
||||
try {
|
||||
|
||||
|
@ -2536,7 +2542,7 @@ public class IndexWriter {
|
|||
segmentInfos.remove(segmentInfos.size()-1);
|
||||
}
|
||||
if (flushDocs)
|
||||
docWriter.abort();
|
||||
docWriter.abort(null);
|
||||
deletePartialSegmentsFile();
|
||||
deleter.checkpoint(segmentInfos, false);
|
||||
|
||||
|
|
|
@ -1887,6 +1887,11 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
throw new IOException("I'm experiencing problems");
|
||||
return input.next(result);
|
||||
}
|
||||
|
||||
public void reset() throws IOException {
|
||||
super.reset();
|
||||
count = 0;
|
||||
}
|
||||
}
|
||||
|
||||
public void testDocumentsWriterExceptions() throws IOException {
|
||||
|
@ -1969,6 +1974,122 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
}
|
||||
}
|
||||
|
||||
public void testDocumentsWriterExceptionThreads() throws IOException {
|
||||
Analyzer analyzer = new Analyzer() {
|
||||
public TokenStream tokenStream(String fieldName, Reader reader) {
|
||||
return new CrashingFilter(fieldName, new WhitespaceTokenizer(reader));
|
||||
}
|
||||
};
|
||||
|
||||
final int NUM_THREAD = 3;
|
||||
final int NUM_ITER = 100;
|
||||
|
||||
for(int i=0;i<2;i++) {
|
||||
MockRAMDirectory dir = new MockRAMDirectory();
|
||||
|
||||
{
|
||||
final IndexWriter writer = new IndexWriter(dir, analyzer);
|
||||
|
||||
final int finalI = i;
|
||||
|
||||
Thread[] threads = new Thread[NUM_THREAD];
|
||||
for(int t=0;t<NUM_THREAD;t++) {
|
||||
threads[t] = new Thread() {
|
||||
public void run() {
|
||||
try {
|
||||
for(int iter=0;iter<NUM_ITER;iter++) {
|
||||
Document doc = new Document();
|
||||
doc.add(new Field("contents", "here are some contents", Field.Store.YES,
|
||||
Field.Index.TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
|
||||
writer.addDocument(doc);
|
||||
writer.addDocument(doc);
|
||||
doc.add(new Field("crash", "this should crash after 4 terms", Field.Store.YES,
|
||||
Field.Index.TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
|
||||
doc.add(new Field("other", "this will not get indexed", Field.Store.YES,
|
||||
Field.Index.TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
|
||||
try {
|
||||
writer.addDocument(doc);
|
||||
fail("did not hit expected exception");
|
||||
} catch (IOException ioe) {
|
||||
}
|
||||
|
||||
if (0 == finalI) {
|
||||
doc = new Document();
|
||||
doc.add(new Field("contents", "here are some contents", Field.Store.YES,
|
||||
Field.Index.TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
|
||||
writer.addDocument(doc);
|
||||
writer.addDocument(doc);
|
||||
}
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
synchronized(this) {
|
||||
System.out.println(Thread.currentThread().getName() + ": ERROR: hit unexpected exception");
|
||||
t.printStackTrace(System.out);
|
||||
}
|
||||
fail();
|
||||
}
|
||||
}
|
||||
};
|
||||
threads[t].start();
|
||||
}
|
||||
|
||||
for(int t=0;t<NUM_THREAD;t++)
|
||||
while (true)
|
||||
try {
|
||||
threads[t].join();
|
||||
break;
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
|
||||
writer.close();
|
||||
}
|
||||
|
||||
IndexReader reader = IndexReader.open(dir);
|
||||
int expected = (3+(1-i)*2)*NUM_THREAD*NUM_ITER;
|
||||
assertEquals(expected, reader.docFreq(new Term("contents", "here")));
|
||||
assertEquals(expected, reader.maxDoc());
|
||||
int numDel = 0;
|
||||
for(int j=0;j<reader.maxDoc();j++) {
|
||||
if (reader.isDeleted(j))
|
||||
numDel++;
|
||||
else
|
||||
reader.document(j);
|
||||
reader.getTermFreqVectors(j);
|
||||
}
|
||||
reader.close();
|
||||
|
||||
assertEquals(NUM_THREAD*NUM_ITER, numDel);
|
||||
|
||||
IndexWriter writer = new IndexWriter(dir, analyzer);
|
||||
writer.setMaxBufferedDocs(10);
|
||||
Document doc = new Document();
|
||||
doc.add(new Field("contents", "here are some contents", Field.Store.YES,
|
||||
Field.Index.TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
|
||||
for(int j=0;j<17;j++)
|
||||
writer.addDocument(doc);
|
||||
writer.optimize();
|
||||
writer.close();
|
||||
|
||||
reader = IndexReader.open(dir);
|
||||
expected += 17-NUM_THREAD*NUM_ITER;
|
||||
assertEquals(expected, reader.docFreq(new Term("contents", "here")));
|
||||
assertEquals(expected, reader.maxDoc());
|
||||
numDel = 0;
|
||||
for(int j=0;j<reader.maxDoc();j++) {
|
||||
if (reader.isDeleted(j))
|
||||
numDel++;
|
||||
else
|
||||
reader.document(j);
|
||||
reader.getTermFreqVectors(j);
|
||||
}
|
||||
reader.close();
|
||||
assertEquals(0, numDel);
|
||||
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
||||
public void testVariableSchema() throws IOException {
|
||||
MockRAMDirectory dir = new MockRAMDirectory();
|
||||
int delID = 0;
|
||||
|
@ -2112,4 +2233,358 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
directory.close();
|
||||
}
|
||||
|
||||
// Used by test cases below
|
||||
private class IndexerThread extends Thread {
|
||||
|
||||
boolean diskFull;
|
||||
Throwable error;
|
||||
AlreadyClosedException ace;
|
||||
IndexWriter writer;
|
||||
boolean noErrors;
|
||||
|
||||
public IndexerThread(IndexWriter writer, boolean noErrors) {
|
||||
this.writer = writer;
|
||||
this.noErrors = noErrors;
|
||||
}
|
||||
|
||||
public void run() {
|
||||
|
||||
final Document doc = new Document();
|
||||
doc.add(new Field("field", "aaa bbb ccc ddd eee fff ggg hhh iii jjj", Field.Store.YES, Field.Index.TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
|
||||
|
||||
int idUpto = 0;
|
||||
int fullCount = 0;
|
||||
|
||||
while(true) {
|
||||
try {
|
||||
writer.updateDocument(new Term("id", ""+(idUpto++)), doc);
|
||||
} catch (IOException ioe) {
|
||||
if (ioe.getMessage().startsWith("fake disk full at") ||
|
||||
ioe.getMessage().equals("now failing on purpose")) {
|
||||
diskFull = true;
|
||||
try {
|
||||
Thread.sleep(1);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
if (fullCount++ >= 5)
|
||||
break;
|
||||
} else {
|
||||
if (noErrors) {
|
||||
System.out.println(Thread.currentThread().getName() + ": ERROR: unexpected IOException:");
|
||||
ioe.printStackTrace(System.out);
|
||||
error = ioe;
|
||||
}
|
||||
break;
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
if (noErrors) {
|
||||
System.out.println(Thread.currentThread().getName() + ": ERROR: unexpected Throwable:");
|
||||
t.printStackTrace(System.out);
|
||||
error = t;
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// LUCENE-1130: make sure we can close() even while
|
||||
// threads are trying to add documents. Strictly
|
||||
// speaking, this isn't valid us of Lucene's APIs, but we
|
||||
// still want to be robust to this case:
|
||||
public void testCloseWithThreads() throws IOException {
|
||||
int NUM_THREADS = 3;
|
||||
|
||||
for(int iter=0;iter<50;iter++) {
|
||||
MockRAMDirectory dir = new MockRAMDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, new WhitespaceAnalyzer());
|
||||
ConcurrentMergeScheduler cms = new ConcurrentMergeScheduler();
|
||||
|
||||
writer.setMergeScheduler(cms);
|
||||
writer.setMaxBufferedDocs(10);
|
||||
writer.setMergeFactor(4);
|
||||
|
||||
IndexerThread[] threads = new IndexerThread[NUM_THREADS];
|
||||
boolean diskFull = false;
|
||||
|
||||
for(int i=0;i<NUM_THREADS;i++)
|
||||
threads[i] = new IndexerThread(writer, false);
|
||||
|
||||
for(int i=0;i<NUM_THREADS;i++)
|
||||
threads[i].start();
|
||||
|
||||
try {
|
||||
Thread.sleep(50);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
|
||||
writer.close(false);
|
||||
|
||||
// Make sure threads that are adding docs are not hung:
|
||||
for(int i=0;i<NUM_THREADS;i++) {
|
||||
while(true) {
|
||||
try {
|
||||
// Without fix for LUCENE-1130: one of the
|
||||
// threads will hang
|
||||
threads[i].join();
|
||||
break;
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
if (threads[i].isAlive())
|
||||
fail("thread seems to be hung");
|
||||
}
|
||||
|
||||
// Quick test to make sure index is not corrupt:
|
||||
IndexReader reader = IndexReader.open(dir);
|
||||
TermDocs tdocs = reader.termDocs(new Term("field", "aaa"));
|
||||
int count = 0;
|
||||
while(tdocs.next()) {
|
||||
count++;
|
||||
}
|
||||
assertTrue(count > 0);
|
||||
reader.close();
|
||||
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
||||
// LUCENE-1130: make sure immeidate disk full on creating
|
||||
// an IndexWriter (hit during DW.ThreadState.init()) is
|
||||
// OK:
|
||||
public void testImmediateDiskFull() throws IOException {
|
||||
MockRAMDirectory dir = new MockRAMDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, new WhitespaceAnalyzer());
|
||||
dir.setMaxSizeInBytes(dir.getRecomputedActualSizeInBytes());
|
||||
writer.setMaxBufferedDocs(2);
|
||||
final Document doc = new Document();
|
||||
doc.add(new Field("field", "aaa bbb ccc ddd eee fff ggg hhh iii jjj", Field.Store.YES, Field.Index.TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
|
||||
try {
|
||||
writer.addDocument(doc);
|
||||
fail("did not hit disk full");
|
||||
} catch (IOException ioe) {
|
||||
}
|
||||
// Without fix for LUCENE-1130: this call will hang:
|
||||
try {
|
||||
writer.addDocument(doc);
|
||||
fail("did not hit disk full");
|
||||
} catch (IOException ioe) {
|
||||
}
|
||||
try {
|
||||
writer.close(false);
|
||||
fail("did not hit disk full");
|
||||
} catch (IOException ioe) {
|
||||
}
|
||||
}
|
||||
|
||||
// LUCENE-1130: make sure immeidate disk full on creating
|
||||
// an IndexWriter (hit during DW.ThreadState.init()), with
|
||||
// multiple threads, is OK:
|
||||
public void testImmediateDiskFullWithThreads() throws IOException {
|
||||
|
||||
int NUM_THREADS = 3;
|
||||
|
||||
for(int iter=0;iter<10;iter++) {
|
||||
MockRAMDirectory dir = new MockRAMDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, new WhitespaceAnalyzer());
|
||||
ConcurrentMergeScheduler cms = new ConcurrentMergeScheduler();
|
||||
// We expect disk full exceptions in the merge threads
|
||||
cms.setSuppressExceptions();
|
||||
writer.setMergeScheduler(cms);
|
||||
writer.setMaxBufferedDocs(2);
|
||||
writer.setMergeFactor(4);
|
||||
dir.setMaxSizeInBytes(4*1024+20*iter);
|
||||
|
||||
IndexerThread[] threads = new IndexerThread[NUM_THREADS];
|
||||
boolean diskFull = false;
|
||||
|
||||
for(int i=0;i<NUM_THREADS;i++)
|
||||
threads[i] = new IndexerThread(writer, true);
|
||||
|
||||
for(int i=0;i<NUM_THREADS;i++)
|
||||
threads[i].start();
|
||||
|
||||
for(int i=0;i<NUM_THREADS;i++) {
|
||||
while(true) {
|
||||
try {
|
||||
// Without fix for LUCENE-1130: one of the
|
||||
// threads will hang
|
||||
threads[i].join();
|
||||
break;
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
if (threads[i].isAlive())
|
||||
fail("thread seems to be hung");
|
||||
else
|
||||
assertTrue("hit unexpected Throwable", threads[i].error == null);
|
||||
}
|
||||
|
||||
try {
|
||||
writer.close(false);
|
||||
} catch (IOException ioe) {
|
||||
}
|
||||
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
||||
// Throws IOException during FieldsWriter.flushDocument and during DocumentsWriter.abort
|
||||
private static class FailOnlyOnAbortOrFlush extends MockRAMDirectory.Failure {
|
||||
public void eval(MockRAMDirectory dir) throws IOException {
|
||||
if (doFail) {
|
||||
StackTraceElement[] trace = new Exception().getStackTrace();
|
||||
for (int i = 0; i < trace.length; i++) {
|
||||
if ("abort".equals(trace[i].getMethodName()) ||
|
||||
"flushDocument".equals(trace[i].getMethodName()))
|
||||
throw new IOException("now failing on purpose");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Runs test, with one thread, using the specific failure
|
||||
// to trigger an IOException
|
||||
public void _testSingleThreadFailure(MockRAMDirectory.Failure failure) throws IOException {
|
||||
MockRAMDirectory dir = new MockRAMDirectory();
|
||||
|
||||
IndexWriter writer = new IndexWriter(dir, new WhitespaceAnalyzer());
|
||||
writer.setMaxBufferedDocs(2);
|
||||
final Document doc = new Document();
|
||||
doc.add(new Field("field", "aaa bbb ccc ddd eee fff ggg hhh iii jjj", Field.Store.YES, Field.Index.TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
|
||||
|
||||
for(int i=0;i<6;i++)
|
||||
writer.addDocument(doc);
|
||||
|
||||
dir.failOn(failure);
|
||||
failure.setDoFail();
|
||||
try {
|
||||
writer.addDocument(doc);
|
||||
writer.addDocument(doc);
|
||||
fail("did not hit exception");
|
||||
} catch (IOException ioe) {
|
||||
}
|
||||
failure.clearDoFail();
|
||||
writer.addDocument(doc);
|
||||
writer.close(false);
|
||||
}
|
||||
|
||||
// Runs test, with multiple threads, using the specific
|
||||
// failure to trigger an IOException
|
||||
public void _testMultipleThreadsFailure(MockRAMDirectory.Failure failure) throws IOException {
|
||||
|
||||
int NUM_THREADS = 3;
|
||||
|
||||
for(int iter=0;iter<5;iter++) {
|
||||
MockRAMDirectory dir = new MockRAMDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, new WhitespaceAnalyzer());
|
||||
ConcurrentMergeScheduler cms = new ConcurrentMergeScheduler();
|
||||
// We expect disk full exceptions in the merge threads
|
||||
cms.setSuppressExceptions();
|
||||
writer.setMergeScheduler(cms);
|
||||
writer.setMaxBufferedDocs(2);
|
||||
writer.setMergeFactor(4);
|
||||
|
||||
IndexerThread[] threads = new IndexerThread[NUM_THREADS];
|
||||
boolean diskFull = false;
|
||||
|
||||
for(int i=0;i<NUM_THREADS;i++)
|
||||
threads[i] = new IndexerThread(writer, true);
|
||||
|
||||
for(int i=0;i<NUM_THREADS;i++)
|
||||
threads[i].start();
|
||||
|
||||
try {
|
||||
Thread.sleep(10);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
|
||||
dir.failOn(failure);
|
||||
failure.setDoFail();
|
||||
|
||||
for(int i=0;i<NUM_THREADS;i++) {
|
||||
while(true) {
|
||||
try {
|
||||
threads[i].join();
|
||||
break;
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
if (threads[i].isAlive())
|
||||
fail("thread seems to be hung");
|
||||
else
|
||||
assertTrue("hit unexpected Throwable", threads[i].error == null);
|
||||
}
|
||||
|
||||
try {
|
||||
writer.close(false);
|
||||
} catch (IOException ioe) {
|
||||
}
|
||||
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
||||
// LUCENE-1130: make sure initial IOException, and then 2nd
|
||||
// IOException during abort(), is OK:
|
||||
public void testIOExceptionDuringAbort() throws IOException {
|
||||
_testSingleThreadFailure(new FailOnlyOnAbortOrFlush());
|
||||
}
|
||||
|
||||
// LUCENE-1130: make sure initial IOException, and then 2nd
|
||||
// IOException during abort(), with multiple threads, is OK:
|
||||
public void testIOExceptionDuringAbortWithThreads() throws IOException {
|
||||
_testMultipleThreadsFailure(new FailOnlyOnAbortOrFlush());
|
||||
}
|
||||
|
||||
// Throws IOException during DocumentsWriter.closeDocStore
|
||||
private static class FailOnlyInCloseDocStore extends MockRAMDirectory.Failure {
|
||||
public void eval(MockRAMDirectory dir) throws IOException {
|
||||
if (doFail) {
|
||||
StackTraceElement[] trace = new Exception().getStackTrace();
|
||||
for (int i = 0; i < trace.length; i++) {
|
||||
if ("closeDocStore".equals(trace[i].getMethodName()))
|
||||
throw new IOException("now failing on purpose");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// LUCENE-1130: test IOException in closeDocStore
|
||||
public void testIOExceptionDuringCloseDocStore() throws IOException {
|
||||
_testSingleThreadFailure(new FailOnlyInCloseDocStore());
|
||||
}
|
||||
|
||||
// LUCENE-1130: test IOException in closeDocStore, with threads
|
||||
public void testIOExceptionDuringCloseDocStoreWithThreads() throws IOException {
|
||||
_testMultipleThreadsFailure(new FailOnlyInCloseDocStore());
|
||||
}
|
||||
|
||||
// Throws IOException during DocumentsWriter.writeSegment
|
||||
private static class FailOnlyInWriteSegment extends MockRAMDirectory.Failure {
|
||||
public void eval(MockRAMDirectory dir) throws IOException {
|
||||
if (doFail) {
|
||||
StackTraceElement[] trace = new Exception().getStackTrace();
|
||||
for (int i = 0; i < trace.length; i++) {
|
||||
if ("writeSegment".equals(trace[i].getMethodName()))
|
||||
throw new IOException("now failing on purpose");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
// LUCENE-1130: test IOException in writeSegment
|
||||
public void testIOExceptionDuringWriteSegment() throws IOException {
|
||||
_testSingleThreadFailure(new FailOnlyInWriteSegment());
|
||||
}
|
||||
|
||||
// LUCENE-1130: test IOException in writeSegment, with threads
|
||||
public void testIOExceptionDuringWriteSegmentWithThreads() throws IOException {
|
||||
_testMultipleThreadsFailure(new FailOnlyInWriteSegment());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.lang.StackTraceElement;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
|
@ -454,7 +453,7 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
|||
String[] startFiles = dir.list();
|
||||
SegmentInfos infos = new SegmentInfos();
|
||||
infos.read(dir);
|
||||
IndexFileDeleter d = new IndexFileDeleter(dir, new KeepOnlyLastCommitDeletionPolicy(), infos, null, null);
|
||||
new IndexFileDeleter(dir, new KeepOnlyLastCommitDeletionPolicy(), infos, null, null);
|
||||
String[] endFiles = dir.list();
|
||||
|
||||
Arrays.sort(startFiles);
|
||||
|
@ -560,8 +559,19 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
|||
}
|
||||
public void eval(MockRAMDirectory dir) throws IOException {
|
||||
if (sawMaybe && !failed) {
|
||||
failed = true;
|
||||
throw new IOException("fail after applyDeletes");
|
||||
boolean seen = false;
|
||||
StackTraceElement[] trace = new Exception().getStackTrace();
|
||||
for (int i = 0; i < trace.length; i++) {
|
||||
if ("applyDeletes".equals(trace[i].getMethodName())) {
|
||||
seen = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (!seen) {
|
||||
// Only fail once we are no longer in applyDeletes
|
||||
failed = true;
|
||||
throw new IOException("fail after applyDeletes");
|
||||
}
|
||||
}
|
||||
if (!failed) {
|
||||
StackTraceElement[] trace = new Exception().getStackTrace();
|
||||
|
@ -740,7 +750,7 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
|||
String[] startFiles = dir.list();
|
||||
SegmentInfos infos = new SegmentInfos();
|
||||
infos.read(dir);
|
||||
IndexFileDeleter d = new IndexFileDeleter(dir, new KeepOnlyLastCommitDeletionPolicy(), infos, null, null);
|
||||
new IndexFileDeleter(dir, new KeepOnlyLastCommitDeletionPolicy(), infos, null, null);
|
||||
String[] endFiles = dir.list();
|
||||
|
||||
if (!Arrays.equals(startFiles, endFiles)) {
|
||||
|
|
|
@ -117,7 +117,6 @@ public class MockRAMDirectory extends RAMDirectory {
|
|||
}
|
||||
|
||||
void maybeThrowIOException() throws IOException {
|
||||
maybeThrowDeterministicException();
|
||||
if (randomIOExceptionRate > 0.0) {
|
||||
int number = Math.abs(randomState.nextInt() % 1000);
|
||||
if (number < randomIOExceptionRate*1000) {
|
||||
|
@ -198,7 +197,7 @@ public class MockRAMDirectory extends RAMDirectory {
|
|||
* RAMOutputStream.BUFFER_SIZE (now 1024) bytes.
|
||||
*/
|
||||
|
||||
final synchronized long getRecomputedActualSizeInBytes() {
|
||||
public final synchronized long getRecomputedActualSizeInBytes() {
|
||||
long size = 0;
|
||||
Iterator it = fileMap.values().iterator();
|
||||
while (it.hasNext())
|
||||
|
@ -245,6 +244,16 @@ public class MockRAMDirectory extends RAMDirectory {
|
|||
* mock.failOn(failure.reset())
|
||||
*/
|
||||
public Failure reset() { return this; }
|
||||
|
||||
protected boolean doFail;
|
||||
|
||||
public void setDoFail() {
|
||||
doFail = true;
|
||||
}
|
||||
|
||||
public void clearDoFail() {
|
||||
doFail = false;
|
||||
}
|
||||
}
|
||||
|
||||
ArrayList failures;
|
||||
|
@ -253,7 +262,7 @@ public class MockRAMDirectory extends RAMDirectory {
|
|||
* add a Failure object to the list of objects to be evaluated
|
||||
* at every potential failure point
|
||||
*/
|
||||
public void failOn(Failure fail) {
|
||||
synchronized public void failOn(Failure fail) {
|
||||
if (failures == null) {
|
||||
failures = new ArrayList();
|
||||
}
|
||||
|
@ -261,10 +270,10 @@ public class MockRAMDirectory extends RAMDirectory {
|
|||
}
|
||||
|
||||
/**
|
||||
* Itterate through the failures list, giving each object a
|
||||
* Iterate through the failures list, giving each object a
|
||||
* chance to throw an IOE
|
||||
*/
|
||||
void maybeThrowDeterministicException() throws IOException {
|
||||
synchronized void maybeThrowDeterministicException() throws IOException {
|
||||
if (failures != null) {
|
||||
for(int i = 0; i < failures.size(); i++) {
|
||||
((Failure)failures.get(i)).eval(this);
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.store;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
|
||||
/**
|
||||
* Used by MockRAMDirectory to create an output stream that
|
||||
|
@ -50,6 +49,11 @@ public class MockRAMOutputStream extends RAMOutputStream {
|
|||
}
|
||||
}
|
||||
|
||||
public void flush() throws IOException {
|
||||
dir.maybeThrowDeterministicException();
|
||||
super.flush();
|
||||
}
|
||||
|
||||
public void writeByte(byte b) throws IOException {
|
||||
singleByte[0] = b;
|
||||
writeBytes(singleByte, 0, 1);
|
||||
|
@ -80,6 +84,8 @@ public class MockRAMOutputStream extends RAMOutputStream {
|
|||
super.writeBytes(b, offset, len);
|
||||
}
|
||||
|
||||
dir.maybeThrowDeterministicException();
|
||||
|
||||
if (first) {
|
||||
// Maybe throw random exception; only do this on first
|
||||
// write to a new file:
|
||||
|
|
Loading…
Reference in New Issue