LUCENE-2053: throw ThreadInterruptedException when the thread is interrupt()d

git-svn-id: https://svn.apache.org/repos/asf/lucene/java/trunk@836158 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2009-11-14 11:33:33 +00:00
parent 0a82d99eff
commit 4422d07e8a
19 changed files with 95 additions and 82 deletions

View File

@ -16,6 +16,14 @@ Changes in backwards compatibility policy
the internal cache implementation for thread safety, before it was the internal cache implementation for thread safety, before it was
declared protected. (Peter Lenahan, Uwe Schindler, Simon Willnauer) declared protected. (Peter Lenahan, Uwe Schindler, Simon Willnauer)
* LUCENE-2053: If you call Thread.interrupt() on a thread inside
Lucene, Lucene will do its best to interrupt the thread. However,
instead of throwing InterruptedException (which is a checked
exception), you'll get an oal.util.ThreadInterruptedException (an
unchecked exception, subclassing RuntimeException). The interrupt
status on the thread is cleared when this exception is thrown.
(Mike McCandless)
Changes in runtime behavior Changes in runtime behavior
* LUCENE-1677: Remove the system property to set SegmentReader class * LUCENE-1677: Remove the system property to set SegmentReader class

View File

@ -42,7 +42,7 @@
<property name="Name" value="Lucene"/> <property name="Name" value="Lucene"/>
<property name="dev.version" value="3.0-dev"/> <property name="dev.version" value="3.0-dev"/>
<property name="version" value="${dev.version}"/> <property name="version" value="${dev.version}"/>
<property name="compatibility.tag" value="lucene_2_9_back_compat_tests_20091111"/> <property name="compatibility.tag" value="lucene_2_9_back_compat_tests_20091114"/>
<property name="spec.version" value="${version}"/> <property name="spec.version" value="${version}"/>
<property name="year" value="2000-${current.year}"/> <property name="year" value="2000-${current.year}"/>
<property name="final.name" value="lucene-${name}-${version}"/> <property name="final.name" value="lucene-${name}-${version}"/>

View File

@ -24,6 +24,7 @@ import java.util.HashMap;
import java.util.Map; import java.util.Map;
import org.apache.lucene.benchmark.byTask.utils.Config; import org.apache.lucene.benchmark.byTask.utils.Config;
import org.apache.lucene.util.ThreadInterruptedException;
import org.xml.sax.Attributes; import org.xml.sax.Attributes;
import org.xml.sax.InputSource; import org.xml.sax.InputSource;
import org.xml.sax.SAXException; import org.xml.sax.SAXException;
@ -66,6 +67,7 @@ public class EnwikiContentSource extends ContentSource {
try { try {
wait(); wait();
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
throw new ThreadInterruptedException(ie);
} }
} }
if (nmde != null) { if (nmde != null) {
@ -127,6 +129,7 @@ public class EnwikiContentSource extends ContentSource {
try { try {
wait(); wait();
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
throw new ThreadInterruptedException(ie);
} }
} }
tuple = tmpTuple; tuple = tmpTuple;

View File

@ -33,6 +33,7 @@ import java.util.zip.GZIPInputStream;
import org.apache.lucene.benchmark.byTask.utils.Config; import org.apache.lucene.benchmark.byTask.utils.Config;
import org.apache.lucene.benchmark.byTask.utils.StringBufferReader; import org.apache.lucene.benchmark.byTask.utils.StringBufferReader;
import org.apache.lucene.util.ThreadInterruptedException;
/** /**
* Implements a {@link ContentSource} over the TREC collection. * Implements a {@link ContentSource} over the TREC collection.
@ -302,10 +303,8 @@ public class TrecContentSource extends ContentSource {
try { try {
docData = htmlParser.parse(docData, name, date, r, null); docData = htmlParser.parse(docData, name, date, r, null);
addDoc(); addDoc();
} catch (InterruptedException e) { } catch (InterruptedException ie) {
IOException ex = new IOException(e.getMessage()); throw new ThreadInterruptedException(ie);
ex.initCause(e);
throw ex;
} }
return docData; return docData;

View File

@ -18,6 +18,7 @@ package org.apache.lucene.index;
*/ */
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.util.ThreadInterruptedException;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;
@ -130,10 +131,7 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
try { try {
wait(); wait();
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
// In 3.0 we will change this to throw throw new ThreadInterruptedException(ie);
// InterruptedException instead
Thread.currentThread().interrupt();
throw new RuntimeException(ie);
} }
} }
} }
@ -200,10 +198,7 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
try { try {
wait(); wait();
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
// In 3.0 we will change this to throw throw new ThreadInterruptedException(ie);
// InterruptedException instead
Thread.currentThread().interrupt();
throw new RuntimeException(ie);
} }
} }
@ -347,9 +342,7 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
// cases: // cases:
Thread.sleep(250); Thread.sleep(250);
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
Thread.currentThread().interrupt(); throw new ThreadInterruptedException(ie);
// In 3.0 this will throw InterruptedException
throw new RuntimeException(ie);
} }
throw new MergePolicy.MergeException(exc, dir); throw new MergePolicy.MergeException(exc, dir);
} }

View File

@ -38,6 +38,7 @@ import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Constants; import org.apache.lucene.util.Constants;
import org.apache.lucene.util.ThreadInterruptedException;
/** /**
* This class accepts multiple added documents and directly * This class accepts multiple added documents and directly
@ -513,10 +514,7 @@ final class DocumentsWriter {
try { try {
wait(); wait();
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
// In 3.0 we will change this to throw throw new ThreadInterruptedException(ie);
// InterruptedException instead
Thread.currentThread().interrupt();
throw new RuntimeException(ie);
} }
} }
@ -851,10 +849,7 @@ final class DocumentsWriter {
try { try {
wait(); wait();
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
// In 3.0 we will change this to throw throw new ThreadInterruptedException(ie);
// InterruptedException instead
Thread.currentThread().interrupt();
throw new RuntimeException(ie);
} }
} }
@ -1108,10 +1103,7 @@ final class DocumentsWriter {
try { try {
wait(); wait();
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
// In 3.0 we will change this to throw throw new ThreadInterruptedException(ie);
// InterruptedException instead
Thread.currentThread().interrupt();
throw new RuntimeException(ie);
} }
} while (!waitQueue.doResume()); } while (!waitQueue.doResume());
} }

View File

@ -28,6 +28,7 @@ import org.apache.lucene.store.LockObtainFailedException;
import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.BufferedIndexInput; import org.apache.lucene.store.BufferedIndexInput;
import org.apache.lucene.util.Constants; import org.apache.lucene.util.Constants;
import org.apache.lucene.util.ThreadInterruptedException;
import java.io.IOException; import java.io.IOException;
import java.io.Closeable; import java.io.Closeable;
@ -144,6 +145,13 @@ import java.util.Map;
synchronize on the <code>IndexWriter</code> instance as synchronize on the <code>IndexWriter</code> instance as
this may cause deadlock; use your own (non-Lucene) objects this may cause deadlock; use your own (non-Lucene) objects
instead. </p> instead. </p>
<p><b>NOTE</b>: If you call
<code>Thread.interrupt()</code> on a thread that's within
IndexWriter, IndexWriter will try to catch this (eg, if
it's in a wait() or Thread.sleep()), and will then throw
the unchecked exception {@link ThreadInterruptedException}
and <b>clear</b> the interrupt status on the thread.</p>
*/ */
/* /*
@ -4506,10 +4514,7 @@ public class IndexWriter implements Closeable {
try { try {
synced.wait(); synced.wait();
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
// In 3.0 we will change this to throw throw new ThreadInterruptedException(ie);
// InterruptedException instead
Thread.currentThread().interrupt();
throw new RuntimeException(ie);
} }
} }
} }
@ -4527,10 +4532,7 @@ public class IndexWriter implements Closeable {
try { try {
wait(1000); wait(1000);
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
// In 3.0 we will change this to throw throw new ThreadInterruptedException(ie);
// InterruptedException instead
Thread.currentThread().interrupt();
throw new RuntimeException(ie);
} }
} }

View File

@ -23,6 +23,7 @@ import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.ChecksumIndexOutput; import org.apache.lucene.store.ChecksumIndexOutput;
import org.apache.lucene.store.ChecksumIndexInput; import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.NoSuchDirectoryException; import org.apache.lucene.store.NoSuchDirectoryException;
import org.apache.lucene.util.ThreadInterruptedException;
import java.io.FileNotFoundException; import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
@ -610,10 +611,7 @@ public final class SegmentInfos extends Vector<SegmentInfo> {
try { try {
Thread.sleep(defaultGenFileRetryPauseMsec); Thread.sleep(defaultGenFileRetryPauseMsec);
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
// In 3.0 we will change this to throw throw new ThreadInterruptedException(ie);
// InterruptedException instead
Thread.currentThread().interrupt();
throw new RuntimeException(ie);
} }
} }

View File

@ -24,6 +24,8 @@ import java.util.Iterator;
import java.util.Map; import java.util.Map;
import java.util.TreeSet; import java.util.TreeSet;
import org.apache.lucene.util.ThreadInterruptedException;
/** /**
* Filter caching singleton. It can be used * Filter caching singleton. It can be used
* to save filters locally for reuse. * to save filters locally for reuse.
@ -193,8 +195,7 @@ public class FilterManager {
try { try {
Thread.sleep(cleanSleepTime); Thread.sleep(cleanSleepTime);
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
Thread.currentThread().interrupt(); throw new ThreadInterruptedException(ie);
throw new RuntimeException(ie);
} }
} }
} }

View File

@ -33,6 +33,7 @@ import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.Term; import org.apache.lucene.index.Term;
import org.apache.lucene.util.NamedThreadFactory; import org.apache.lucene.util.NamedThreadFactory;
import org.apache.lucene.util.PriorityQueue; import org.apache.lucene.util.PriorityQueue;
import org.apache.lucene.util.ThreadInterruptedException;
/** Implements parallel search over a set of <code>Searchables</code>. /** Implements parallel search over a set of <code>Searchables</code>.
* *
@ -186,11 +187,8 @@ public class ParallelMultiSearcher extends MultiSearcher {
if (e.getCause() instanceof IOException) if (e.getCause() instanceof IOException)
throw (IOException) e.getCause(); throw (IOException) e.getCause();
throw new RuntimeException(e.getCause()); throw new RuntimeException(e.getCause());
} catch (InterruptedException e) { } catch (InterruptedException ie) {
Thread.currentThread().interrupt(); throw new ThreadInterruptedException(ie);
// In 3.0 we will change this to throw
// InterruptedException instead
throw new RuntimeException(e);
} }
} }
} }

View File

@ -20,6 +20,7 @@ package org.apache.lucene.search;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.util.ThreadInterruptedException;
/** /**
* The {@link TimeLimitingCollector} is used to timeout search requests that * The {@link TimeLimitingCollector} is used to timeout search requests that
@ -78,8 +79,7 @@ public class TimeLimitingCollector extends Collector {
try { try {
Thread.sleep( resolution ); Thread.sleep( resolution );
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
Thread.currentThread().interrupt(); throw new ThreadInterruptedException(ie);
throw new RuntimeException(ie);
} }
} }
} }

View File

@ -24,6 +24,7 @@ import java.io.RandomAccessFile;
import java.security.MessageDigest; import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException; import java.security.NoSuchAlgorithmException;
import org.apache.lucene.util.ThreadInterruptedException;
import org.apache.lucene.util.Constants; import org.apache.lucene.util.Constants;
/** /**
@ -317,10 +318,7 @@ public abstract class FSDirectory extends Directory {
// Pause 5 msec // Pause 5 msec
Thread.sleep(5); Thread.sleep(5);
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
// In 3.0 we will change this to throw throw new ThreadInterruptedException(ie);
// InterruptedException instead
Thread.currentThread().interrupt();
throw new RuntimeException(ie);
} }
} }
} }

View File

@ -17,6 +17,7 @@ package org.apache.lucene.store;
* limitations under the License. * limitations under the License.
*/ */
import org.apache.lucene.util.ThreadInterruptedException;
import java.io.IOException; import java.io.IOException;
/** An interprocess mutex lock. /** An interprocess mutex lock.
@ -88,11 +89,8 @@ public abstract class Lock {
} }
try { try {
Thread.sleep(LOCK_POLL_INTERVAL); Thread.sleep(LOCK_POLL_INTERVAL);
} catch (InterruptedException e) { } catch (InterruptedException ie) {
// In 3.0 we will change this to throw throw new ThreadInterruptedException(ie);
// InterruptedException instead
Thread.currentThread().interrupt();
throw new IOException(e.toString());
} }
locked = obtain(); locked = obtain();
} }

View File

@ -22,6 +22,7 @@ import java.io.FileNotFoundException;
import java.io.Serializable; import java.io.Serializable;
import java.util.HashMap; import java.util.HashMap;
import java.util.Set; import java.util.Set;
import org.apache.lucene.util.ThreadInterruptedException;
/** /**
* A memory-resident {@link Directory} implementation. Locking * A memory-resident {@link Directory} implementation. Locking
@ -124,10 +125,7 @@ public class RAMDirectory extends Directory implements Serializable {
try { try {
Thread.sleep(0, 1); Thread.sleep(0, 1);
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
// In 3.0 we will change this to throw throw new ThreadInterruptedException(ie);
// InterruptedException instead
Thread.currentThread().interrupt();
throw new RuntimeException(ie);
} }
ts2 = System.currentTimeMillis(); ts2 = System.currentTimeMillis();
} while(ts1 == ts2); } while(ts1 == ts2);

View File

@ -0,0 +1,30 @@
package org.apache.lucene.util;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/**
* Thrown by lucene on detecting that Thread.interrupt() had
* been called. Unlike Java's InterruptedException, this
* exception is not checked..
*/
public final class ThreadInterruptedException extends RuntimeException {
public ThreadInterruptedException(InterruptedException ie) {
super(ie);
}
}

View File

@ -36,7 +36,7 @@ import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy;
import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.TestIndexWriter; import org.apache.lucene.index.TestIndexWriter;
import org.apache.lucene.index.SnapshotDeletionPolicy; import org.apache.lucene.index.SnapshotDeletionPolicy;
import org.apache.lucene.util.ThreadInterruptedException;
import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil; import org.apache.lucene.util._TestUtil;
@ -141,8 +141,7 @@ public class TestSnapshotDeletionPolicy extends LuceneTestCase
try { try {
Thread.sleep(1); Thread.sleep(1);
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
Thread.currentThread().interrupt(); throw new ThreadInterruptedException(ie);
throw new RuntimeException(ie);
} }
} }
} }

View File

@ -66,6 +66,7 @@ import org.apache.lucene.store.SingleInstanceLockFactory;
import org.apache.lucene.util.UnicodeUtil; import org.apache.lucene.util.UnicodeUtil;
import org.apache.lucene.util._TestUtil; import org.apache.lucene.util._TestUtil;
import org.apache.lucene.util.Version; import org.apache.lucene.util.Version;
import org.apache.lucene.util.ThreadInterruptedException;
public class TestIndexWriter extends LuceneTestCase { public class TestIndexWriter extends LuceneTestCase {
public TestIndexWriter(String name) { public TestIndexWriter(String name) {
@ -2216,8 +2217,7 @@ public class TestIndexWriter extends LuceneTestCase {
try { try {
Thread.sleep(1); Thread.sleep(1);
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
Thread.currentThread().interrupt(); throw new ThreadInterruptedException(ie);
throw new RuntimeException(ie);
} }
if (fullCount++ >= 5) if (fullCount++ >= 5)
break; break;
@ -4385,18 +4385,13 @@ public class TestIndexWriter extends LuceneTestCase {
w.addDocument(doc); w.addDocument(doc);
w.commit(); w.commit();
} }
} catch (RuntimeException re) { } catch (ThreadInterruptedException re) {
Throwable e = re.getCause(); Throwable e = re.getCause();
if (e instanceof InterruptedException) { assertTrue(e instanceof InterruptedException);
// Make sure IW restored interrupted bit
if (!interrupted()) { // Make sure IW cleared the interrupted bit
System.out.println("FAILED; InterruptedException hit but thread.interrupted() was false"); if (interrupted()) {
e.printStackTrace(System.out); System.out.println("FAILED; InterruptedException hit but thread.interrupted() was true");
failed = true;
break;
}
} else {
System.out.println("FAILED; unexpected exception");
e.printStackTrace(System.out); e.printStackTrace(System.out);
failed = true; failed = true;
break; break;

View File

@ -38,6 +38,7 @@ import org.apache.lucene.store.MockRAMDirectory;
import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil; import org.apache.lucene.util._TestUtil;
import org.apache.lucene.util.ThreadInterruptedException;
public class TestIndexWriterReader extends LuceneTestCase { public class TestIndexWriterReader extends LuceneTestCase {
static PrintStream infoStream; static PrintStream infoStream;
@ -360,7 +361,7 @@ public class TestIndexWriterReader extends LuceneTestCase {
try { try {
threads[i].join(); threads[i].join();
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
Thread.currentThread().interrupt(); throw new ThreadInterruptedException(ie);
} }
} }
} }
@ -402,7 +403,7 @@ public class TestIndexWriterReader extends LuceneTestCase {
try { try {
threads[i].join(); threads[i].join();
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
Thread.currentThread().interrupt(); throw new ThreadInterruptedException(ie);
} }
} }

View File

@ -32,6 +32,7 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.store.RAMDirectory; import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.Version; import org.apache.lucene.util.Version;
import org.apache.lucene.util.ThreadInterruptedException;
/** /**
* Tests the {@link TimeLimitingCollector}. This test checks (1) search * Tests the {@link TimeLimitingCollector}. This test checks (1) search
@ -328,8 +329,7 @@ public class TestTimeLimitingCollector extends LuceneTestCase {
try { try {
Thread.sleep(slowdown); Thread.sleep(slowdown);
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
Thread.currentThread().interrupt(); throw new ThreadInterruptedException(ie);
throw new RuntimeException(ie);
} }
} }
assert docId >= 0: " base=" + docBase + " doc=" + doc; assert docId >= 0: " base=" + docBase + " doc=" + doc;