mirror of https://github.com/apache/lucene.git
LUCENE-1573: don't ignore, or enter spinloop, when Thread.interrupt() is called
git-svn-id: https://svn.apache.org/repos/asf/lucene/java/trunk@759556 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
c73712d1bb
commit
7aa012ca27
|
@ -101,6 +101,12 @@ Bug fixes
|
|||
7. LUCENE-1327: Fix TermSpans#skipTo() to behave as specified in javadocs
|
||||
of Terms#skipTo(). (Michael Busch)
|
||||
|
||||
8. LUCENE-1573: Do not ignore InterruptedException (caused by
|
||||
Thread.interrupt()) nor enter deadlock/spin loop. Now, an interrupt
|
||||
will cause a RuntimeException to be thrown. In 3.0 we will change
|
||||
public APIs to throw InterruptedException. (Jeremy Volkman vai
|
||||
Mike McCandless)
|
||||
|
||||
New features
|
||||
|
||||
1. LUCENE-1411: Added expert API to open an IndexWriter on a prior
|
||||
|
|
|
@ -611,7 +611,7 @@ public class CheckIndex {
|
|||
This tool exits with exit code 1 if the index cannot be opened or has any
|
||||
corruption, else 0.
|
||||
*/
|
||||
public static void main(String[] args) throws IOException {
|
||||
public static void main(String[] args) throws IOException, InterruptedException {
|
||||
|
||||
boolean doFix = false;
|
||||
List onlySegments = new ArrayList();
|
||||
|
@ -695,13 +695,7 @@ public class CheckIndex {
|
|||
System.out.println("WARNING: " + result.totLoseDocCount + " documents will be lost\n");
|
||||
System.out.println("NOTE: will write new segments file in 5 seconds; this will remove " + result.totLoseDocCount + " docs from the index. THIS IS YOUR LAST CHANCE TO CTRL+C!");
|
||||
for(int s=0;s<5;s++) {
|
||||
try {
|
||||
Thread.sleep(1000);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
s--;
|
||||
continue;
|
||||
}
|
||||
Thread.sleep(1000);
|
||||
System.out.println(" " + (5-s) + "...");
|
||||
}
|
||||
System.out.println("Writing...");
|
||||
|
|
|
@ -129,10 +129,15 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
|
|||
|
||||
try {
|
||||
wait();
|
||||
} catch (InterruptedException e) {
|
||||
} catch (InterruptedException ie) {
|
||||
// In 3.0 we will change this to throw
|
||||
// InterruptedException instead
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RuntimeException(ie);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private synchronized int mergeThreadCount() {
|
||||
int count = 0;
|
||||
final int numThreads = mergeThreads.size();
|
||||
|
@ -185,29 +190,42 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
|
|||
// deterministic assignment of segment names
|
||||
writer.mergeInit(merge);
|
||||
|
||||
synchronized(this) {
|
||||
while (mergeThreadCount() >= maxThreadCount) {
|
||||
if (verbose())
|
||||
message(" too many merge threads running; stalling...");
|
||||
try {
|
||||
wait();
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
boolean success = false;
|
||||
try {
|
||||
synchronized(this) {
|
||||
final MergeThread merger;
|
||||
while (mergeThreadCount() >= maxThreadCount) {
|
||||
if (verbose())
|
||||
message(" too many merge threads running; stalling...");
|
||||
try {
|
||||
wait();
|
||||
} catch (InterruptedException ie) {
|
||||
// In 3.0 we will change this to throw
|
||||
// InterruptedException instead
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RuntimeException(ie);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (verbose())
|
||||
message(" consider merge " + merge.segString(dir));
|
||||
if (verbose())
|
||||
message(" consider merge " + merge.segString(dir));
|
||||
|
||||
assert mergeThreadCount() < maxThreadCount;
|
||||
assert mergeThreadCount() < maxThreadCount;
|
||||
|
||||
// OK to spawn a new merge thread to handle this
|
||||
// merge:
|
||||
final MergeThread merger = getMergeThread(writer, merge);
|
||||
mergeThreads.add(merger);
|
||||
if (verbose())
|
||||
message(" launch new thread [" + merger.getName() + "]");
|
||||
merger.start();
|
||||
// OK to spawn a new merge thread to handle this
|
||||
// merge:
|
||||
merger = getMergeThread(writer, merge);
|
||||
mergeThreads.add(merger);
|
||||
if (verbose())
|
||||
message(" launch new thread [" + merger.getName() + "]");
|
||||
|
||||
merger.start();
|
||||
success = true;
|
||||
}
|
||||
} finally {
|
||||
if (!success) {
|
||||
writer.mergeFinish(merge);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -509,8 +509,11 @@ final class DocumentsWriter {
|
|||
while(!allThreadsIdle()) {
|
||||
try {
|
||||
wait();
|
||||
} catch (InterruptedException e) {
|
||||
} catch (InterruptedException ie) {
|
||||
// In 3.0 we will change this to throw
|
||||
// InterruptedException instead
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RuntimeException(ie);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -837,8 +840,11 @@ final class DocumentsWriter {
|
|||
while (!closed && ((state != null && !state.isIdle) || pauseThreads != 0 || flushPending || aborting)) {
|
||||
try {
|
||||
wait();
|
||||
} catch (InterruptedException e) {
|
||||
} catch (InterruptedException ie) {
|
||||
// In 3.0 we will change this to throw
|
||||
// InterruptedException instead
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RuntimeException(ie);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1083,8 +1089,11 @@ final class DocumentsWriter {
|
|||
do {
|
||||
try {
|
||||
wait();
|
||||
} catch (InterruptedException e) {
|
||||
} catch (InterruptedException ie) {
|
||||
// In 3.0 we will change this to throw
|
||||
// InterruptedException instead
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RuntimeException(ie);
|
||||
}
|
||||
} while (!waitQueue.doResume());
|
||||
}
|
||||
|
|
|
@ -4147,25 +4147,20 @@ public class IndexWriter {
|
|||
}
|
||||
} finally {
|
||||
synchronized(this) {
|
||||
try {
|
||||
mergeFinish(merge);
|
||||
|
||||
mergeFinish(merge);
|
||||
|
||||
if (!success) {
|
||||
if (infoStream != null)
|
||||
message("hit exception during merge");
|
||||
if (merge.info != null && !segmentInfos.contains(merge.info))
|
||||
deleter.refresh(merge.info.name);
|
||||
}
|
||||
|
||||
// This merge (and, generally, any change to the
|
||||
// segments) may now enable new merges, so we call
|
||||
// merge policy & update pending merges.
|
||||
if (success && !merge.isAborted() && !closed && !closing)
|
||||
updatePendingMerges(merge.maxNumSegmentsOptimize, merge.optimize);
|
||||
} finally {
|
||||
runningMerges.remove(merge);
|
||||
if (!success) {
|
||||
if (infoStream != null)
|
||||
message("hit exception during merge");
|
||||
if (merge.info != null && !segmentInfos.contains(merge.info))
|
||||
deleter.refresh(merge.info.name);
|
||||
}
|
||||
|
||||
// This merge (and, generally, any change to the
|
||||
// segments) may now enable new merges, so we call
|
||||
// merge policy & update pending merges.
|
||||
if (success && !merge.isAborted() && !closed && !closing)
|
||||
updatePendingMerges(merge.maxNumSegmentsOptimize, merge.optimize);
|
||||
}
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
|
@ -4234,7 +4229,6 @@ public class IndexWriter {
|
|||
} finally {
|
||||
if (!success) {
|
||||
mergeFinish(merge);
|
||||
runningMerges.remove(merge);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -4439,6 +4433,8 @@ public class IndexWriter {
|
|||
mergingSegments.remove(merge.info);
|
||||
merge.registerDone = false;
|
||||
}
|
||||
|
||||
runningMerges.remove(merge);
|
||||
}
|
||||
|
||||
/** Does the actual (time-consuming) work of the merge,
|
||||
|
@ -4700,7 +4696,10 @@ public class IndexWriter {
|
|||
try {
|
||||
synced.wait();
|
||||
} catch (InterruptedException ie) {
|
||||
continue;
|
||||
// In 3.0 we will change this to throw
|
||||
// InterruptedException instead
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RuntimeException(ie);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -4731,23 +4730,29 @@ public class IndexWriter {
|
|||
try {
|
||||
Thread.sleep(100);
|
||||
} catch (InterruptedException ie) {
|
||||
// In 3.0 we will change this to throw
|
||||
// InterruptedException instead
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RuntimeException(ie);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private synchronized void doWait() {
|
||||
// NOTE: the callers of this method should in theory
|
||||
// be able to do simply wait(), but, as a defense
|
||||
// against thread timing hazards where notifyAll()
|
||||
// falls to be called, we wait for at most 1 second
|
||||
// and then return so caller can check if wait
|
||||
// conditions are satisified:
|
||||
try {
|
||||
// NOTE: the callers of this method should in theory
|
||||
// be able to do simply wait(), but, as a defense
|
||||
// against thread timing hazards where notifyAll()
|
||||
// falls to be called, we wait for at most 1 second
|
||||
// and then return so caller can check if wait
|
||||
// conditions are satisified:
|
||||
wait(1000);
|
||||
} catch (InterruptedException ie) {
|
||||
// In 3.0 we will change this to throw
|
||||
// InterruptedException instead
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RuntimeException(ie);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -4819,6 +4824,13 @@ public class IndexWriter {
|
|||
|
||||
deleter.incRef(toSync, false);
|
||||
myChangeCount = changeCount;
|
||||
|
||||
Iterator it = toSync.files(directory, false).iterator();
|
||||
while(it.hasNext()) {
|
||||
String fileName = (String) it.next();
|
||||
assert directory.fileExists(fileName): "file " + fileName + " does not exist";
|
||||
}
|
||||
|
||||
} finally {
|
||||
resumeAddIndexes();
|
||||
}
|
||||
|
|
|
@ -613,8 +613,11 @@ final class SegmentInfos extends Vector {
|
|||
}
|
||||
try {
|
||||
Thread.sleep(defaultGenFileRetryPauseMsec);
|
||||
} catch (InterruptedException e) {
|
||||
// will retry
|
||||
} catch (InterruptedException ie) {
|
||||
// In 3.0 we will change this to throw
|
||||
// InterruptedException instead
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RuntimeException(ie);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -195,8 +195,9 @@ public class FilterManager {
|
|||
// take a nap
|
||||
try {
|
||||
Thread.sleep(cleanSleepTime);
|
||||
} catch (InterruptedException e) {
|
||||
// just keep going
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RuntimeException(ie);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -77,7 +77,10 @@ public class ParallelMultiSearcher extends MultiSearcher {
|
|||
try {
|
||||
msta[i].join();
|
||||
} catch (InterruptedException ie) {
|
||||
; // TODO: what should we do with this???
|
||||
// In 3.0 we will change this to throw
|
||||
// InterruptedException instead
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RuntimeException(ie);
|
||||
}
|
||||
IOException ioe = msta[i].getIOException();
|
||||
if (ioe == null) {
|
||||
|
@ -130,7 +133,10 @@ public class ParallelMultiSearcher extends MultiSearcher {
|
|||
try {
|
||||
msta[i].join();
|
||||
} catch (InterruptedException ie) {
|
||||
; // TODO: what should we do with this???
|
||||
// In 3.0 we will change this to throw
|
||||
// InterruptedException instead
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RuntimeException(ie);
|
||||
}
|
||||
IOException ioe = msta[i].getIOException();
|
||||
if (ioe == null) {
|
||||
|
|
|
@ -72,21 +72,14 @@ public class TimeLimitedCollector extends MultiReaderHitCollector {
|
|||
}
|
||||
|
||||
public void run() {
|
||||
boolean interrupted = false;
|
||||
try {
|
||||
while( true ) {
|
||||
// TODO: Use System.nanoTime() when Lucene moves to Java SE 5.
|
||||
time += resolution;
|
||||
try {
|
||||
Thread.sleep( resolution );
|
||||
} catch( final InterruptedException e ) {
|
||||
interrupted = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
finally {
|
||||
if( interrupted ) {
|
||||
while( true ) {
|
||||
// TODO: Use System.nanoTime() when Lucene moves to Java SE 5.
|
||||
time += resolution;
|
||||
try {
|
||||
Thread.sleep( resolution );
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RuntimeException(ie);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -538,7 +538,10 @@ public class FSDirectory extends Directory {
|
|||
// Pause 5 msec
|
||||
Thread.sleep(5);
|
||||
} catch (InterruptedException ie) {
|
||||
// In 3.0 we will change this to throw
|
||||
// InterruptedException instead
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RuntimeException(ie);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -91,6 +91,8 @@ public abstract class Lock {
|
|||
try {
|
||||
Thread.sleep(LOCK_POLL_INTERVAL);
|
||||
} catch (InterruptedException e) {
|
||||
// In 3.0 we will change this to throw
|
||||
// InterruptedException instead
|
||||
throw new IOException(e.toString());
|
||||
}
|
||||
locked = obtain();
|
||||
|
|
|
@ -150,7 +150,12 @@ public class RAMDirectory extends Directory implements Serializable {
|
|||
do {
|
||||
try {
|
||||
Thread.sleep(0, 1);
|
||||
} catch (InterruptedException e) {}
|
||||
} catch (InterruptedException ie) {
|
||||
// In 3.0 we will change this to throw
|
||||
// InterruptedException instead
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RuntimeException(ie);
|
||||
}
|
||||
ts2 = System.currentTimeMillis();
|
||||
} while(ts1 == ts2);
|
||||
|
||||
|
|
|
@ -49,7 +49,7 @@ public class TestSnapshotDeletionPolicy extends LuceneTestCase
|
|||
{
|
||||
public static final String INDEX_PATH = "test.snapshots";
|
||||
|
||||
public void testSnapshotDeletionPolicy() throws IOException {
|
||||
public void testSnapshotDeletionPolicy() throws Exception {
|
||||
File dir = new File(System.getProperty("tempDir"), INDEX_PATH);
|
||||
try {
|
||||
Directory fsDir = FSDirectory.getDirectory(dir);
|
||||
|
@ -63,7 +63,7 @@ public class TestSnapshotDeletionPolicy extends LuceneTestCase
|
|||
runTest(dir2);
|
||||
}
|
||||
|
||||
public void testReuseAcrossWriters() throws IOException {
|
||||
public void testReuseAcrossWriters() throws Exception {
|
||||
Directory dir = new MockRAMDirectory();
|
||||
|
||||
SnapshotDeletionPolicy dp = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy());
|
||||
|
@ -98,7 +98,7 @@ public class TestSnapshotDeletionPolicy extends LuceneTestCase
|
|||
dir.close();
|
||||
}
|
||||
|
||||
private void runTest(Directory dir) throws IOException {
|
||||
private void runTest(Directory dir) throws Exception {
|
||||
// Run for ~7 seconds
|
||||
final long stopTime = System.currentTimeMillis() + 7000;
|
||||
|
||||
|
@ -125,6 +125,7 @@ public class TestSnapshotDeletionPolicy extends LuceneTestCase
|
|||
Thread.sleep(1);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RuntimeException(ie);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -136,20 +137,12 @@ public class TestSnapshotDeletionPolicy extends LuceneTestCase
|
|||
// backups:
|
||||
while(System.currentTimeMillis() < stopTime) {
|
||||
backupIndex(dir, dp);
|
||||
try {
|
||||
Thread.sleep(20);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
Thread.sleep(20);
|
||||
if (!t.isAlive())
|
||||
break;
|
||||
}
|
||||
|
||||
try {
|
||||
t.join();
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
t.join();
|
||||
|
||||
// Add one more document to force writer to commit a
|
||||
// final segment, so deletion policy has a chance to
|
||||
|
@ -169,7 +162,7 @@ public class TestSnapshotDeletionPolicy extends LuceneTestCase
|
|||
* backup; instead, it reads every byte of every file
|
||||
* just to test that the files indeed exist and are
|
||||
* readable even while the index is changing. */
|
||||
public void backupIndex(Directory dir, SnapshotDeletionPolicy dp) throws IOException {
|
||||
public void backupIndex(Directory dir, SnapshotDeletionPolicy dp) throws Exception {
|
||||
// To backup an index we first take a snapshot:
|
||||
try {
|
||||
copyFiles(dir, (IndexCommit) dp.snapshot());
|
||||
|
@ -181,7 +174,7 @@ public class TestSnapshotDeletionPolicy extends LuceneTestCase
|
|||
}
|
||||
}
|
||||
|
||||
private void copyFiles(Directory dir, IndexCommit cp) throws IOException {
|
||||
private void copyFiles(Directory dir, IndexCommit cp) throws Exception {
|
||||
|
||||
// While we hold the snapshot, and nomatter how long
|
||||
// we take to do the backup, the IndexWriter will
|
||||
|
@ -202,7 +195,7 @@ public class TestSnapshotDeletionPolicy extends LuceneTestCase
|
|||
|
||||
byte[] buffer = new byte[4096];
|
||||
|
||||
private void readFile(Directory dir, String name) throws IOException {
|
||||
private void readFile(Directory dir, String name) throws Exception {
|
||||
IndexInput input = dir.openInput(name);
|
||||
try {
|
||||
long size = dir.fileLength(name);
|
||||
|
@ -221,11 +214,7 @@ public class TestSnapshotDeletionPolicy extends LuceneTestCase
|
|||
// make sure we are exercising the fact that the
|
||||
// IndexWriter should not delete this file even when I
|
||||
// take my time reading it.
|
||||
try {
|
||||
Thread.sleep(1);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
Thread.sleep(1);
|
||||
} finally {
|
||||
input.close();
|
||||
}
|
||||
|
|
|
@ -23,8 +23,6 @@ import org.apache.lucene.analysis.SimpleAnalyzer;
|
|||
import org.apache.lucene.analysis.standard.StandardAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.Field.Index;
|
||||
import org.apache.lucene.document.Field.Store;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.FSDirectory;
|
||||
import org.apache.lucene.store.RAMDirectory;
|
||||
|
@ -132,13 +130,13 @@ public class TestIndexModifier extends LuceneTestCase {
|
|||
return doc;
|
||||
}
|
||||
|
||||
public void testIndexWithThreads() throws IOException {
|
||||
public void testIndexWithThreads() throws Exception {
|
||||
testIndexInternal(0);
|
||||
testIndexInternal(10);
|
||||
testIndexInternal(50);
|
||||
}
|
||||
|
||||
private void testIndexInternal(int maxWait) throws IOException {
|
||||
private void testIndexInternal(int maxWait) throws Exception {
|
||||
final boolean create = true;
|
||||
//Directory rd = new RAMDirectory();
|
||||
// work on disk to make sure potential lock problems are tested:
|
||||
|
@ -155,11 +153,7 @@ public class TestIndexModifier extends LuceneTestCase {
|
|||
IndexThread thread2 = new IndexThread(index, maxWait, 2);
|
||||
thread2.start();
|
||||
while(thread1.isAlive() || thread2.isAlive()) {
|
||||
try {
|
||||
Thread.sleep(100);
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
Thread.sleep(100);
|
||||
}
|
||||
index.optimize();
|
||||
int added = thread1.added + thread2.added;
|
||||
|
@ -253,12 +247,13 @@ class IndexThread extends Thread {
|
|||
deleted++;
|
||||
}
|
||||
if (maxWait > 0) {
|
||||
rand = random.nextInt(maxWait);
|
||||
//System.out.println("waiting " + rand + "ms");
|
||||
try {
|
||||
rand = random.nextInt(maxWait);
|
||||
//System.out.println("waiting " + rand + "ms");
|
||||
Thread.sleep(rand);
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RuntimeException(ie);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -750,7 +750,7 @@ public class TestIndexReader extends LuceneTestCase
|
|||
_TestUtil.rmDir(dirFile);
|
||||
}
|
||||
|
||||
public void testLastModified() throws IOException {
|
||||
public void testLastModified() throws Exception {
|
||||
assertFalse(IndexReader.indexExists("there_is_no_such_index"));
|
||||
final File fileDir = new File(System.getProperty("tempDir"), "testIndex");
|
||||
for(int i=0;i<2;i++) {
|
||||
|
@ -776,14 +776,7 @@ public class TestIndexReader extends LuceneTestCase
|
|||
reader.close();
|
||||
// modify index and check version has been
|
||||
// incremented:
|
||||
while(true) {
|
||||
try {
|
||||
Thread.sleep(1000);
|
||||
break;
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
Thread.sleep(1000);
|
||||
|
||||
writer = new IndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
|
||||
addDocumentWithFields(writer);
|
||||
|
|
|
@ -770,11 +770,9 @@ public class TestIndexReaderReopen extends LuceneTestCase {
|
|||
}
|
||||
readersToClose.add(refreshed);
|
||||
}
|
||||
try {
|
||||
synchronized(this) {
|
||||
wait(1000);
|
||||
}
|
||||
} catch (InterruptedException e) {}
|
||||
synchronized(this) {
|
||||
wait(1000);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -789,11 +787,9 @@ public class TestIndexReaderReopen extends LuceneTestCase {
|
|||
TestIndexReader.assertIndexEquals(c.newReader, c.refreshedReader);
|
||||
}
|
||||
|
||||
try {
|
||||
synchronized(this) {
|
||||
wait(100);
|
||||
}
|
||||
} catch (InterruptedException e) {}
|
||||
synchronized(this) {
|
||||
wait(100);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -806,9 +802,7 @@ public class TestIndexReaderReopen extends LuceneTestCase {
|
|||
}
|
||||
|
||||
synchronized(this) {
|
||||
try {
|
||||
wait(15000);
|
||||
} catch(InterruptedException e) {}
|
||||
wait(15000);
|
||||
}
|
||||
|
||||
for (int i = 0; i < n; i++) {
|
||||
|
@ -819,13 +813,11 @@ public class TestIndexReaderReopen extends LuceneTestCase {
|
|||
|
||||
for (int i = 0; i < n; i++) {
|
||||
if (threads[i] != null) {
|
||||
try {
|
||||
threads[i].join();
|
||||
if (threads[i].error != null) {
|
||||
String msg = "Error occurred in thread " + threads[i].getName() + ":\n" + threads[i].error.getMessage();
|
||||
fail(msg);
|
||||
}
|
||||
} catch (InterruptedException e) {}
|
||||
threads[i].join();
|
||||
if (threads[i].error != null) {
|
||||
String msg = "Error occurred in thread " + threads[i].getName() + ":\n" + threads[i].error.getMessage();
|
||||
fail(msg);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -2010,7 +2010,7 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
}
|
||||
}
|
||||
|
||||
public void testDocumentsWriterExceptionThreads() throws IOException {
|
||||
public void testDocumentsWriterExceptionThreads() throws Exception {
|
||||
Analyzer analyzer = new Analyzer() {
|
||||
public TokenStream tokenStream(String fieldName, Reader reader) {
|
||||
return new CrashingFilter(fieldName, new WhitespaceTokenizer(reader));
|
||||
|
@ -2070,13 +2070,7 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
}
|
||||
|
||||
for(int t=0;t<NUM_THREAD;t++)
|
||||
while (true)
|
||||
try {
|
||||
threads[t].join();
|
||||
break;
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
threads[t].join();
|
||||
|
||||
writer.close();
|
||||
}
|
||||
|
@ -2128,7 +2122,7 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
}
|
||||
}
|
||||
|
||||
public void testVariableSchema() throws IOException {
|
||||
public void testVariableSchema() throws Exception {
|
||||
MockRAMDirectory dir = new MockRAMDirectory();
|
||||
int delID = 0;
|
||||
for(int i=0;i<20;i++) {
|
||||
|
@ -2249,14 +2243,7 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
t1.start();
|
||||
|
||||
writer.close(false);
|
||||
while(true) {
|
||||
try {
|
||||
t1.join();
|
||||
break;
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
t1.join();
|
||||
|
||||
// Make sure reader can read
|
||||
IndexReader reader = IndexReader.open(directory);
|
||||
|
@ -2309,6 +2296,7 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
Thread.sleep(1);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RuntimeException(ie);
|
||||
}
|
||||
if (fullCount++ >= 5)
|
||||
break;
|
||||
|
@ -2337,7 +2325,7 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
// 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 {
|
||||
public void testCloseWithThreads() throws Exception {
|
||||
int NUM_THREADS = 3;
|
||||
|
||||
for(int iter=0;iter<20;iter++) {
|
||||
|
@ -2362,11 +2350,7 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
|
||||
boolean done = false;
|
||||
while(!done) {
|
||||
try {
|
||||
Thread.sleep(100);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
Thread.sleep(100);
|
||||
for(int i=0;i<NUM_THREADS;i++)
|
||||
// only stop when at least one thread has added a doc
|
||||
if (threads[i].addCount > 0) {
|
||||
|
@ -2379,16 +2363,9 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
|
||||
// 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();
|
||||
}
|
||||
}
|
||||
// Without fix for LUCENE-1130: one of the
|
||||
// threads will hang
|
||||
threads[i].join();
|
||||
if (threads[i].isAlive())
|
||||
fail("thread seems to be hung");
|
||||
}
|
||||
|
@ -2438,7 +2415,7 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
// LUCENE-1130: make sure immediate disk full on creating
|
||||
// an IndexWriter (hit during DW.ThreadState.init()), with
|
||||
// multiple threads, is OK:
|
||||
public void testImmediateDiskFullWithThreads() throws IOException {
|
||||
public void testImmediateDiskFullWithThreads() throws Exception {
|
||||
|
||||
int NUM_THREADS = 3;
|
||||
|
||||
|
@ -2462,20 +2439,10 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
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);
|
||||
// Without fix for LUCENE-1130: one of the
|
||||
// threads will hang
|
||||
threads[i].join();
|
||||
assertTrue("hit unexpected Throwable", threads[i].error == null);
|
||||
}
|
||||
|
||||
try {
|
||||
|
@ -2538,7 +2505,7 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
|
||||
// Runs test, with multiple threads, using the specific
|
||||
// failure to trigger an IOException
|
||||
public void _testMultipleThreadsFailure(MockRAMDirectory.Failure failure) throws IOException {
|
||||
public void _testMultipleThreadsFailure(MockRAMDirectory.Failure failure) throws Exception {
|
||||
|
||||
int NUM_THREADS = 3;
|
||||
|
||||
|
@ -2560,28 +2527,14 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
for(int i=0;i<NUM_THREADS;i++)
|
||||
threads[i].start();
|
||||
|
||||
try {
|
||||
Thread.sleep(10);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
Thread.sleep(10);
|
||||
|
||||
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);
|
||||
threads[i].join();
|
||||
assertTrue("hit unexpected Throwable", threads[i].error == null);
|
||||
}
|
||||
|
||||
boolean success = false;
|
||||
|
@ -2622,13 +2575,13 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
|
||||
// LUCENE-1130: make sure initial IOException, and then 2nd
|
||||
// IOException during abort(), with multiple threads, is OK:
|
||||
public void testIOExceptionDuringAbortWithThreads() throws IOException {
|
||||
public void testIOExceptionDuringAbortWithThreads() throws Exception {
|
||||
_testMultipleThreadsFailure(new FailOnlyOnAbortOrFlush(false));
|
||||
}
|
||||
|
||||
// LUCENE-1130: make sure initial IOException, and then 2nd
|
||||
// IOException during abort(), with multiple threads, is OK:
|
||||
public void testIOExceptionDuringAbortWithThreadsOnlyOnce() throws IOException {
|
||||
public void testIOExceptionDuringAbortWithThreadsOnlyOnce() throws Exception {
|
||||
_testMultipleThreadsFailure(new FailOnlyOnAbortOrFlush(true));
|
||||
}
|
||||
|
||||
|
@ -2663,12 +2616,12 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
}
|
||||
|
||||
// LUCENE-1130: test IOException in closeDocStore, with threads
|
||||
public void testIOExceptionDuringCloseDocStoreWithThreads() throws IOException {
|
||||
public void testIOExceptionDuringCloseDocStoreWithThreads() throws Exception {
|
||||
_testMultipleThreadsFailure(new FailOnlyInCloseDocStore(false));
|
||||
}
|
||||
|
||||
// LUCENE-1130: test IOException in closeDocStore, with threads
|
||||
public void testIOExceptionDuringCloseDocStoreWithThreadsOnlyOnce() throws IOException {
|
||||
public void testIOExceptionDuringCloseDocStoreWithThreadsOnlyOnce() throws Exception {
|
||||
_testMultipleThreadsFailure(new FailOnlyInCloseDocStore(true));
|
||||
}
|
||||
|
||||
|
@ -2703,12 +2656,12 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
}
|
||||
|
||||
// LUCENE-1130: test IOException in writeSegment, with threads
|
||||
public void testIOExceptionDuringWriteSegmentWithThreads() throws IOException {
|
||||
public void testIOExceptionDuringWriteSegmentWithThreads() throws Exception {
|
||||
_testMultipleThreadsFailure(new FailOnlyInWriteSegment(false));
|
||||
}
|
||||
|
||||
// LUCENE-1130: test IOException in writeSegment, with threads
|
||||
public void testIOExceptionDuringWriteSegmentWithThreadsOnlyOnce() throws IOException {
|
||||
public void testIOExceptionDuringWriteSegmentWithThreadsOnlyOnce() throws Exception {
|
||||
_testMultipleThreadsFailure(new FailOnlyInWriteSegment(true));
|
||||
}
|
||||
|
||||
|
@ -3828,13 +3781,9 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
threads[i].start();
|
||||
}
|
||||
|
||||
void joinThreads() {
|
||||
void joinThreads() throws Exception {
|
||||
for(int i=0;i<NUM_THREADS;i++)
|
||||
try {
|
||||
threads[i].join();
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
threads[i].join();
|
||||
}
|
||||
|
||||
void close(boolean doWait) throws Throwable {
|
||||
|
@ -3995,11 +3944,7 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
CommitAndAddIndexes3 c = new CommitAndAddIndexes3(NUM_COPY);
|
||||
c.launchThreads(-1);
|
||||
|
||||
try {
|
||||
Thread.sleep(500);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
Thread.sleep(500);
|
||||
|
||||
// Close w/o first stopping/joining the threads
|
||||
c.close(false);
|
||||
|
@ -4020,11 +3965,7 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
CommitAndAddIndexes3 c = new CommitAndAddIndexes3(NUM_COPY);
|
||||
c.launchThreads(-1);
|
||||
|
||||
try {
|
||||
Thread.sleep(500);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
Thread.sleep(500);
|
||||
|
||||
// Close w/o first stopping/joining the threads
|
||||
c.didClose = true;
|
||||
|
@ -4317,4 +4258,100 @@ public class TestIndexWriter extends LuceneTestCase
|
|||
dir2.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
private class IndexerThreadInterrupt extends Thread {
|
||||
volatile boolean failed;
|
||||
volatile boolean finish;
|
||||
public void run() {
|
||||
RAMDirectory dir = new RAMDirectory();
|
||||
IndexWriter w = null;
|
||||
while(!finish) {
|
||||
try {
|
||||
//IndexWriter.unlock(dir);
|
||||
w = new IndexWriter(dir, new WhitespaceAnalyzer(), IndexWriter.MaxFieldLength.UNLIMITED);
|
||||
((ConcurrentMergeScheduler) w.getMergeScheduler()).setSuppressExceptions();
|
||||
//w.setInfoStream(System.out);
|
||||
w.setMaxBufferedDocs(2);
|
||||
w.setMergeFactor(2);
|
||||
Document doc = new Document();
|
||||
doc.add(new Field("field", "some text contents", Field.Store.YES, Field.Index.ANALYZED));
|
||||
for(int i=0;i<100;i++) {
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
}
|
||||
} catch (RuntimeException re) {
|
||||
Throwable e = re.getCause();
|
||||
if (e instanceof InterruptedException) {
|
||||
// Make sure IW restored interrupted bit
|
||||
if (!interrupted()) {
|
||||
System.out.println("FAILED; InterruptedException hit but thread.interrupted() was false");
|
||||
e.printStackTrace(System.out);
|
||||
failed = true;
|
||||
break;
|
||||
}
|
||||
} else {
|
||||
System.out.println("FAILED; unexpected exception");
|
||||
e.printStackTrace(System.out);
|
||||
failed = true;
|
||||
break;
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
System.out.println("FAILED; unexpected exception");
|
||||
t.printStackTrace(System.out);
|
||||
failed = true;
|
||||
break;
|
||||
} finally {
|
||||
try {
|
||||
// Clear interrupt if pending
|
||||
synchronized(this) {
|
||||
interrupted();
|
||||
if (w != null) {
|
||||
w.close();
|
||||
}
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
System.out.println("FAILED; unexpected exception during close");
|
||||
t.printStackTrace(System.out);
|
||||
failed = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!failed) {
|
||||
try {
|
||||
_TestUtil.checkIndex(dir);
|
||||
} catch (Exception e) {
|
||||
failed = true;
|
||||
System.out.println("CheckIndex FAILED: unexpected exception");
|
||||
e.printStackTrace(System.out);
|
||||
}
|
||||
try {
|
||||
IndexReader r = IndexReader.open(dir);
|
||||
//System.out.println("doc count=" + r.numDocs());
|
||||
r.close();
|
||||
} catch (Exception e) {
|
||||
failed = true;
|
||||
System.out.println("IndexReader.open FAILED: unexpected exception");
|
||||
e.printStackTrace(System.out);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testThreadInterruptDeadlock() throws Exception {
|
||||
IndexerThreadInterrupt t = new IndexerThreadInterrupt();
|
||||
t.setDaemon(true);
|
||||
t.start();
|
||||
for(int i=0;i<100;i++) {
|
||||
Thread.sleep(1);
|
||||
synchronized(t) {
|
||||
t.interrupt();
|
||||
}
|
||||
}
|
||||
t.finish = true;
|
||||
t.interrupt();
|
||||
t.join();
|
||||
assertFalse(t.failed);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -475,7 +475,7 @@ public class TestPayloads extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testThreadSafety() throws IOException {
|
||||
public void testThreadSafety() throws Exception {
|
||||
rnd = newRandom();
|
||||
final int numThreads = 5;
|
||||
final int numDocs = 50;
|
||||
|
@ -505,9 +505,7 @@ public class TestPayloads extends LuceneTestCase {
|
|||
}
|
||||
|
||||
for (int i = 0; i < numThreads; i++) {
|
||||
try {
|
||||
ingesters[i].join();
|
||||
} catch (InterruptedException e) {}
|
||||
ingesters[i].join();
|
||||
}
|
||||
writer.close();
|
||||
IndexReader reader = IndexReader.open(dir);
|
||||
|
|
|
@ -57,7 +57,7 @@ public class TestMultiThreadTermVectors extends LuceneTestCase {
|
|||
|
||||
}
|
||||
|
||||
public void test() {
|
||||
public void test() throws Exception {
|
||||
|
||||
IndexReader reader = null;
|
||||
|
||||
|
@ -83,7 +83,7 @@ public class TestMultiThreadTermVectors extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testTermPositionVectors(final IndexReader reader, int threadCount) {
|
||||
public void testTermPositionVectors(final IndexReader reader, int threadCount) throws Exception {
|
||||
MultiThreadTermVectorsReader[] mtr = new MultiThreadTermVectorsReader[threadCount];
|
||||
for (int i = 0; i < threadCount; i++) {
|
||||
mtr[i] = new MultiThreadTermVectorsReader();
|
||||
|
@ -94,7 +94,6 @@ public class TestMultiThreadTermVectors extends LuceneTestCase {
|
|||
/** run until all threads finished */
|
||||
int threadsAlive = mtr.length;
|
||||
while (threadsAlive > 0) {
|
||||
try {
|
||||
//System.out.println("Threads alive");
|
||||
Thread.sleep(10);
|
||||
threadsAlive = mtr.length;
|
||||
|
@ -104,10 +103,7 @@ public class TestMultiThreadTermVectors extends LuceneTestCase {
|
|||
}
|
||||
|
||||
threadsAlive--;
|
||||
|
||||
}
|
||||
|
||||
} catch (InterruptedException ie) {}
|
||||
}
|
||||
}
|
||||
|
||||
long totalTime = 0L;
|
||||
|
|
|
@ -248,18 +248,18 @@ public class TestTimeLimitedCollector extends LuceneTestCase {
|
|||
/**
|
||||
* Test correctness with multiple searching threads.
|
||||
*/
|
||||
public void testSearchMultiThreaded() {
|
||||
public void testSearchMultiThreaded() throws Exception {
|
||||
doTestMultiThreads(false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test correctness with multiple searching threads.
|
||||
*/
|
||||
public void testTimeoutMultiThreaded() {
|
||||
public void testTimeoutMultiThreaded() throws Exception {
|
||||
doTestMultiThreads(true);
|
||||
}
|
||||
|
||||
private void doTestMultiThreads(final boolean withTimeout) {
|
||||
private void doTestMultiThreads(final boolean withTimeout) throws Exception {
|
||||
Thread [] threadArray = new Thread[N_THREADS];
|
||||
final BitSet success = new BitSet(N_THREADS);
|
||||
for( int i = 0; i < threadArray.length; ++i ) {
|
||||
|
@ -280,16 +280,8 @@ public class TestTimeLimitedCollector extends LuceneTestCase {
|
|||
for( int i = 0; i < threadArray.length; ++i ) {
|
||||
threadArray[i].start();
|
||||
}
|
||||
boolean interrupted = false;
|
||||
for( int i = 0; i < threadArray.length; ++i ) {
|
||||
try {
|
||||
threadArray[i].join();
|
||||
} catch (InterruptedException e) {
|
||||
interrupted = true;
|
||||
}
|
||||
}
|
||||
if (interrupted) {
|
||||
Thread.currentThread().interrupt();
|
||||
threadArray[i].join();
|
||||
}
|
||||
assertEquals("some threads failed!", N_THREADS,success.cardinality());
|
||||
}
|
||||
|
@ -314,9 +306,9 @@ public class TestTimeLimitedCollector extends LuceneTestCase {
|
|||
if( slowdown > 0 ) {
|
||||
try {
|
||||
Thread.sleep(slowdown);
|
||||
}
|
||||
catch(InterruptedException x) {
|
||||
System.out.println("caught " + x);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RuntimeException(ie);
|
||||
}
|
||||
}
|
||||
assert docId >= 0: " base=" + docBase + " doc=" + doc;
|
||||
|
|
|
@ -326,7 +326,7 @@ public class TestLockFactory extends LuceneTestCase {
|
|||
// Verify: do stress test, by opening IndexReaders and
|
||||
// IndexWriters over & over in 2 threads and making sure
|
||||
// no unexpected exceptions are raised:
|
||||
public void testStressLocks() throws IOException {
|
||||
public void testStressLocks() throws Exception {
|
||||
_testStressLocks(null, "index.TestLockFactory6");
|
||||
}
|
||||
|
||||
|
@ -334,11 +334,11 @@ public class TestLockFactory extends LuceneTestCase {
|
|||
// IndexWriters over & over in 2 threads and making sure
|
||||
// no unexpected exceptions are raised, but use
|
||||
// NativeFSLockFactory:
|
||||
public void testStressLocksNativeFSLockFactory() throws IOException {
|
||||
public void testStressLocksNativeFSLockFactory() throws Exception {
|
||||
_testStressLocks(new NativeFSLockFactory("index.TestLockFactory7"), "index.TestLockFactory7");
|
||||
}
|
||||
|
||||
public void _testStressLocks(LockFactory lockFactory, String indexDirName) throws IOException {
|
||||
public void _testStressLocks(LockFactory lockFactory, String indexDirName) throws Exception {
|
||||
FSDirectory fs1 = FSDirectory.getDirectory(indexDirName, lockFactory);
|
||||
|
||||
// First create a 1 doc index:
|
||||
|
@ -353,10 +353,7 @@ public class TestLockFactory extends LuceneTestCase {
|
|||
searcher.start();
|
||||
|
||||
while(writer.isAlive() || searcher.isAlive()) {
|
||||
try {
|
||||
Thread.sleep(1000);
|
||||
} catch (InterruptedException e) {
|
||||
}
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
|
||||
assertTrue("IndexWriter hit unexpected exceptions", !writer.hitException);
|
||||
|
|
Loading…
Reference in New Issue