You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by mi...@apache.org on 2009/11/14 12:33:34 UTC

svn commit: r836158 - in /lucene/java/trunk: ./ contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/ src/java/org/apache/lucene/index/ src/java/org/apache/lucene/search/ src/java/org/apache/lucene/store/ src/java/org/apache/lucene/util/...

Author: mikemccand
Date: Sat Nov 14 11:33:33 2009
New Revision: 836158

URL: http://svn.apache.org/viewvc?rev=836158&view=rev
Log:
LUCENE-2053: throw ThreadInterruptedException when the thread is interrupt()d

Added:
    lucene/java/trunk/src/java/org/apache/lucene/util/ThreadInterruptedException.java   (with props)
Modified:
    lucene/java/trunk/CHANGES.txt
    lucene/java/trunk/common-build.xml
    lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/EnwikiContentSource.java
    lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/TrecContentSource.java
    lucene/java/trunk/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
    lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java
    lucene/java/trunk/src/java/org/apache/lucene/search/FilterManager.java
    lucene/java/trunk/src/java/org/apache/lucene/search/ParallelMultiSearcher.java
    lucene/java/trunk/src/java/org/apache/lucene/search/TimeLimitingCollector.java
    lucene/java/trunk/src/java/org/apache/lucene/store/FSDirectory.java
    lucene/java/trunk/src/java/org/apache/lucene/store/Lock.java
    lucene/java/trunk/src/java/org/apache/lucene/store/RAMDirectory.java
    lucene/java/trunk/src/test/org/apache/lucene/TestSnapshotDeletionPolicy.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriterReader.java
    lucene/java/trunk/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java

Modified: lucene/java/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/java/trunk/CHANGES.txt?rev=836158&r1=836157&r2=836158&view=diff
==============================================================================
--- lucene/java/trunk/CHANGES.txt (original)
+++ lucene/java/trunk/CHANGES.txt Sat Nov 14 11:33:33 2009
@@ -16,6 +16,14 @@
   the internal cache implementation for thread safety, before it was
   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
 
 * LUCENE-1677: Remove the system property to set SegmentReader class

Modified: lucene/java/trunk/common-build.xml
URL: http://svn.apache.org/viewvc/lucene/java/trunk/common-build.xml?rev=836158&r1=836157&r2=836158&view=diff
==============================================================================
--- lucene/java/trunk/common-build.xml (original)
+++ lucene/java/trunk/common-build.xml Sat Nov 14 11:33:33 2009
@@ -42,7 +42,7 @@
   <property name="Name" value="Lucene"/>
   <property name="dev.version" value="3.0-dev"/>
   <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="year" value="2000-${current.year}"/>
   <property name="final.name" value="lucene-${name}-${version}"/>

Modified: lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/EnwikiContentSource.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/EnwikiContentSource.java?rev=836158&r1=836157&r2=836158&view=diff
==============================================================================
--- lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/EnwikiContentSource.java (original)
+++ lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/EnwikiContentSource.java Sat Nov 14 11:33:33 2009
@@ -24,6 +24,7 @@
 import java.util.Map;
 
 import org.apache.lucene.benchmark.byTask.utils.Config;
+import org.apache.lucene.util.ThreadInterruptedException;
 import org.xml.sax.Attributes;
 import org.xml.sax.InputSource;
 import org.xml.sax.SAXException;
@@ -66,6 +67,7 @@
           try {
             wait();
           } catch (InterruptedException ie) {
+            throw new ThreadInterruptedException(ie);
           }
         }
         if (nmde != null) {
@@ -127,6 +129,7 @@
                 try {
                   wait();
                 } catch (InterruptedException ie) {
+                  throw new ThreadInterruptedException(ie);
                 }
               }
               tuple = tmpTuple;

Modified: lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/TrecContentSource.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/TrecContentSource.java?rev=836158&r1=836157&r2=836158&view=diff
==============================================================================
--- lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/TrecContentSource.java (original)
+++ lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/TrecContentSource.java Sat Nov 14 11:33:33 2009
@@ -33,6 +33,7 @@
 
 import org.apache.lucene.benchmark.byTask.utils.Config;
 import org.apache.lucene.benchmark.byTask.utils.StringBufferReader;
+import org.apache.lucene.util.ThreadInterruptedException;
 
 /**
  * Implements a {@link ContentSource} over the TREC collection.
@@ -302,10 +303,8 @@
     try {
       docData = htmlParser.parse(docData, name, date, r, null);
       addDoc();
-    } catch (InterruptedException e) {
-      IOException ex = new IOException(e.getMessage());
-      ex.initCause(e);
-      throw ex;
+    } catch (InterruptedException ie) {
+      throw new ThreadInterruptedException(ie);
     }
 
     return docData;

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java?rev=836158&r1=836157&r2=836158&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java Sat Nov 14 11:33:33 2009
@@ -18,6 +18,7 @@
  */
 
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.ThreadInterruptedException;
 
 import java.io.IOException;
 import java.util.List;
@@ -130,10 +131,7 @@
       try {
         wait();
       } catch (InterruptedException ie) {
-        // In 3.0 we will change this to throw
-        // InterruptedException instead
-        Thread.currentThread().interrupt();
-        throw new RuntimeException(ie);
+        throw new ThreadInterruptedException(ie);
       }
     }
   }
@@ -200,10 +198,7 @@
             try {
               wait();
             } catch (InterruptedException ie) {
-              // In 3.0 we will change this to throw
-              // InterruptedException instead
-              Thread.currentThread().interrupt();
-              throw new RuntimeException(ie);
+              throw new ThreadInterruptedException(ie);
             }
           }
 
@@ -347,9 +342,7 @@
       // cases:
       Thread.sleep(250);
     } catch (InterruptedException ie) {
-      Thread.currentThread().interrupt();
-      // In 3.0 this will throw InterruptedException
-      throw new RuntimeException(ie);
+      throw new ThreadInterruptedException(ie);
     }
     throw new MergePolicy.MergeException(exc, dir);
   }

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=836158&r1=836157&r2=836158&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java Sat Nov 14 11:33:33 2009
@@ -38,6 +38,7 @@
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.ThreadInterruptedException;
 
 /**
  * This class accepts multiple added documents and directly
@@ -513,10 +514,7 @@
       try {
         wait();
       } catch (InterruptedException ie) {
-        // In 3.0 we will change this to throw
-        // InterruptedException instead
-        Thread.currentThread().interrupt();
-        throw new RuntimeException(ie);
+        throw new ThreadInterruptedException(ie);
       }
     }
 
@@ -851,10 +849,7 @@
       try {
         wait();
       } catch (InterruptedException ie) {
-        // In 3.0 we will change this to throw
-        // InterruptedException instead
-        Thread.currentThread().interrupt();
-        throw new RuntimeException(ie);
+        throw new ThreadInterruptedException(ie);
       }
     }
 
@@ -1108,10 +1103,7 @@
       try {
         wait();
       } catch (InterruptedException ie) {
-        // In 3.0 we will change this to throw
-        // InterruptedException instead
-        Thread.currentThread().interrupt();
-        throw new RuntimeException(ie);
+        throw new ThreadInterruptedException(ie);
       }
     } while (!waitQueue.doResume());
   }

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java?rev=836158&r1=836157&r2=836158&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java Sat Nov 14 11:33:33 2009
@@ -28,6 +28,7 @@
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.BufferedIndexInput;
 import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.ThreadInterruptedException;
 
 import java.io.IOException;
 import java.io.Closeable;
@@ -144,6 +145,13 @@
   synchronize on the <code>IndexWriter</code> instance as
   this may cause deadlock; use your own (non-Lucene) objects
   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 @@
             try {
               synced.wait();
             } catch (InterruptedException ie) {
-              // In 3.0 we will change this to throw
-              // InterruptedException instead
-              Thread.currentThread().interrupt();
-              throw new RuntimeException(ie);
+              throw new ThreadInterruptedException(ie);
             }
         }
       }
@@ -4527,10 +4532,7 @@
     try {
       wait(1000);
     } catch (InterruptedException ie) {
-      // In 3.0 we will change this to throw
-      // InterruptedException instead
-      Thread.currentThread().interrupt();
-      throw new RuntimeException(ie);
+      throw new ThreadInterruptedException(ie);
     }
   }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java?rev=836158&r1=836157&r2=836158&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java Sat Nov 14 11:33:33 2009
@@ -23,6 +23,7 @@
 import org.apache.lucene.store.ChecksumIndexOutput;
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.NoSuchDirectoryException;
+import org.apache.lucene.util.ThreadInterruptedException;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -610,10 +611,7 @@
             try {
               Thread.sleep(defaultGenFileRetryPauseMsec);
             } catch (InterruptedException ie) {
-              // In 3.0 we will change this to throw
-              // InterruptedException instead
-              Thread.currentThread().interrupt();
-              throw new RuntimeException(ie);
+              throw new ThreadInterruptedException(ie);
             }
           }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/FilterManager.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/FilterManager.java?rev=836158&r1=836157&r2=836158&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/FilterManager.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/FilterManager.java Sat Nov 14 11:33:33 2009
@@ -24,6 +24,8 @@
 import java.util.Map;
 import java.util.TreeSet;
 
+import org.apache.lucene.util.ThreadInterruptedException;
+
 /**
  * Filter caching singleton.  It can be used 
  * to save filters locally for reuse.
@@ -193,8 +195,7 @@
         try {
           Thread.sleep(cleanSleepTime);
         } catch (InterruptedException ie) {
-          Thread.currentThread().interrupt();
-          throw new RuntimeException(ie);
+          throw new ThreadInterruptedException(ie);
         }
       }
     }

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/ParallelMultiSearcher.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/ParallelMultiSearcher.java?rev=836158&r1=836157&r2=836158&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/ParallelMultiSearcher.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/ParallelMultiSearcher.java Sat Nov 14 11:33:33 2009
@@ -33,6 +33,7 @@
 import org.apache.lucene.index.Term;
 import org.apache.lucene.util.NamedThreadFactory;
 import org.apache.lucene.util.PriorityQueue;
+import org.apache.lucene.util.ThreadInterruptedException;
 
 /** Implements parallel search over a set of <code>Searchables</code>.
  *
@@ -186,11 +187,8 @@
         if (e.getCause() instanceof IOException)
           throw (IOException) e.getCause();
         throw new RuntimeException(e.getCause());
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        // In 3.0 we will change this to throw
-        // InterruptedException instead
-        throw new RuntimeException(e);
+      } catch (InterruptedException ie) {
+        throw new ThreadInterruptedException(ie);
       }
     }
   }

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/TimeLimitingCollector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/TimeLimitingCollector.java?rev=836158&r1=836157&r2=836158&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/TimeLimitingCollector.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/TimeLimitingCollector.java Sat Nov 14 11:33:33 2009
@@ -20,6 +20,7 @@
 import java.io.IOException;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.util.ThreadInterruptedException;
 
 /**
  * The {@link TimeLimitingCollector} is used to timeout search requests that
@@ -78,8 +79,7 @@
         try {
           Thread.sleep( resolution );
         } catch (InterruptedException ie) {
-          Thread.currentThread().interrupt();
-          throw new RuntimeException(ie);
+          throw new ThreadInterruptedException(ie);
         }
       }
     }

Modified: lucene/java/trunk/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/FSDirectory.java?rev=836158&r1=836157&r2=836158&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/FSDirectory.java Sat Nov 14 11:33:33 2009
@@ -24,6 +24,7 @@
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
 
+import org.apache.lucene.util.ThreadInterruptedException;
 import org.apache.lucene.util.Constants;
 
 /**
@@ -317,10 +318,7 @@
           // 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);
+          throw new ThreadInterruptedException(ie);
         }
       }
     }

Modified: lucene/java/trunk/src/java/org/apache/lucene/store/Lock.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/Lock.java?rev=836158&r1=836157&r2=836158&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/Lock.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/Lock.java Sat Nov 14 11:33:33 2009
@@ -17,6 +17,7 @@
  * limitations under the License.
  */
 
+import org.apache.lucene.util.ThreadInterruptedException;
 import java.io.IOException;
 
 /** An interprocess mutex lock.
@@ -88,11 +89,8 @@
       }
       try {
         Thread.sleep(LOCK_POLL_INTERVAL);
-      } catch (InterruptedException e) {
-        // In 3.0 we will change this to throw
-        // InterruptedException instead
-        Thread.currentThread().interrupt();
-        throw new IOException(e.toString());
+      } catch (InterruptedException ie) {
+        throw new ThreadInterruptedException(ie);
       }
       locked = obtain();
     }

Modified: lucene/java/trunk/src/java/org/apache/lucene/store/RAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/RAMDirectory.java?rev=836158&r1=836157&r2=836158&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/RAMDirectory.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/RAMDirectory.java Sat Nov 14 11:33:33 2009
@@ -22,6 +22,7 @@
 import java.io.Serializable;
 import java.util.HashMap;
 import java.util.Set;
+import org.apache.lucene.util.ThreadInterruptedException;
 
 /**
  * A memory-resident {@link Directory} implementation.  Locking
@@ -124,10 +125,7 @@
       try {
         Thread.sleep(0, 1);
       } catch (InterruptedException ie) {
-        // In 3.0 we will change this to throw
-        // InterruptedException instead
-        Thread.currentThread().interrupt();
-        throw new RuntimeException(ie);
+        throw new ThreadInterruptedException(ie);
       }
       ts2 = System.currentTimeMillis();
     } while(ts1 == ts2);

Added: lucene/java/trunk/src/java/org/apache/lucene/util/ThreadInterruptedException.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/ThreadInterruptedException.java?rev=836158&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/ThreadInterruptedException.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/ThreadInterruptedException.java Sat Nov 14 11:33:33 2009
@@ -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);
+  }
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/util/ThreadInterruptedException.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/java/trunk/src/test/org/apache/lucene/TestSnapshotDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/TestSnapshotDeletionPolicy.java?rev=836158&r1=836157&r2=836158&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/TestSnapshotDeletionPolicy.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/TestSnapshotDeletionPolicy.java Sat Nov 14 11:33:33 2009
@@ -36,7 +36,7 @@
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.TestIndexWriter;
 import org.apache.lucene.index.SnapshotDeletionPolicy;
-
+import org.apache.lucene.util.ThreadInterruptedException;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
 
@@ -141,8 +141,7 @@
             try {
               Thread.sleep(1);
             } catch (InterruptedException ie) {
-              Thread.currentThread().interrupt();
-              throw new RuntimeException(ie);
+              throw new ThreadInterruptedException(ie);
             }
           }
         }

Modified: lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=836158&r1=836157&r2=836158&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java Sat Nov 14 11:33:33 2009
@@ -66,6 +66,7 @@
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util._TestUtil;
 import org.apache.lucene.util.Version;
+import org.apache.lucene.util.ThreadInterruptedException;
 
 public class TestIndexWriter extends LuceneTestCase {
     public TestIndexWriter(String name) {
@@ -2216,8 +2217,7 @@
             try {
               Thread.sleep(1);
             } catch (InterruptedException ie) {
-              Thread.currentThread().interrupt();
-              throw new RuntimeException(ie);
+              throw new ThreadInterruptedException(ie);
             }
             if (fullCount++ >= 5)
               break;
@@ -4385,18 +4385,13 @@
             w.addDocument(doc);
             w.commit();
           }
-        } catch (RuntimeException re) {
+        } catch (ThreadInterruptedException 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");
+          assertTrue(e instanceof InterruptedException);
+          
+          // Make sure IW cleared the interrupted bit
+          if (interrupted()) {
+            System.out.println("FAILED; InterruptedException hit but thread.interrupted() was true");
             e.printStackTrace(System.out);
             failed = true;
             break;

Modified: lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriterReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriterReader.java?rev=836158&r1=836157&r2=836158&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriterReader.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriterReader.java Sat Nov 14 11:33:33 2009
@@ -38,6 +38,7 @@
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
+import org.apache.lucene.util.ThreadInterruptedException;
 
 public class TestIndexWriterReader extends LuceneTestCase {
   static PrintStream infoStream;
@@ -360,7 +361,7 @@
         try {
           threads[i].join();
         } catch (InterruptedException ie) {
-          Thread.currentThread().interrupt();
+          throw new ThreadInterruptedException(ie);
         }
     }
   }
@@ -402,7 +403,7 @@
         try {
           threads[i].join();
         } catch (InterruptedException ie) {
-          Thread.currentThread().interrupt();
+          throw new ThreadInterruptedException(ie);
         }
     }
 

Modified: lucene/java/trunk/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java?rev=836158&r1=836157&r2=836158&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java Sat Nov 14 11:33:33 2009
@@ -32,6 +32,7 @@
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.Version;
+import org.apache.lucene.util.ThreadInterruptedException;
 
 /**
  * Tests the {@link TimeLimitingCollector}.  This test checks (1) search
@@ -328,8 +329,7 @@
         try {
           Thread.sleep(slowdown);
         } catch (InterruptedException ie) {
-          Thread.currentThread().interrupt();
-          throw new RuntimeException(ie);
+          throw new ThreadInterruptedException(ie);
         }
       }
       assert docId >= 0: " base=" + docBase + " doc=" + doc;