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/03/18 11:22:52 UTC

svn commit: r755535 - in /lucene/java/trunk: contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java src/java/org/apache/lucene/index/IndexWriter.java src/test/org/apache/lucene/index/TestIndexWriter.java

Author: mikemccand
Date: Wed Mar 18 10:22:51 2009
New Revision: 755535

URL: http://svn.apache.org/viewvc?rev=755535&view=rev
Log:
improve IndexWriter.rollback javadoc src/java/org/apache/lucene/index/IndexWriter.java

Modified:
    lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java
    lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java

Modified: lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java?rev=755535&r1=755534&r2=755535&view=diff
==============================================================================
--- lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java (original)
+++ lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java Wed Mar 18 10:22:51 2009
@@ -62,6 +62,8 @@
     super(runData);
   }
 
+  static boolean first = true;
+
   public int doLogic() throws Exception {
     int res = 0;
     boolean closeReader = false;
@@ -101,6 +103,17 @@
           hits = searcher.search(q, numHits);
         }
         //System.out.println("q=" + q + ":" + hits.totalHits + " total hits"); 
+        if (first) {
+          System.out.println("NUMHITS=" + hits.totalHits);
+
+          for(int i=0;i<10;i++) {
+            if (i >= hits.totalHits) {
+              break;
+            }
+            System.out.println("  " + i + ": score=" + hits.scoreDocs[i].score + " doc=" + hits.scoreDocs[i].doc);
+          }
+          first = false;
+        }
 
         if (withTraverse()) {
           final ScoreDoc[] scoreDocs = hits.scoreDocs;

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=755535&r1=755534&r2=755535&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 Wed Mar 18 10:22:51 2009
@@ -2870,11 +2870,12 @@
    * Close the <code>IndexWriter</code> without committing
    * any changes that have occurred since the last commit
    * (or since it was opened, if commit hasn't been called).
-   * This removes any temporary files that had been
-   * created, after which the state of the index will be the
-   * same as it was when this writer was first opened.  This
-   * can only be called when this IndexWriter was opened
-   * with <code>autoCommit=false</code>.  This also clears a
+   * This removes any temporary files that had been created,
+   * after which the state of the index will be the same as
+   * it was when commit() was last called or when this
+   * writer was first opened.  This can only be called when
+   * this IndexWriter was opened with
+   * <code>autoCommit=false</code>.  This also clears a
    * previous call to {@link #prepareCommit}.
    * @throws IllegalStateException if this is called when
    *  the writer was opened with <code>autoCommit=true</code>.

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=755535&r1=755534&r2=755535&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 Wed Mar 18 10:22:51 2009
@@ -66,7 +66,7 @@
  */
 public class TestIndexWriter extends LuceneTestCase
 {
-    public void testDocCount() throws IOException
+    public void xxxtestDocCount() throws IOException
     {
         Directory dir = new RAMDirectory();
 
@@ -151,7 +151,7 @@
       either all or none of the incoming documents were in
       fact added.
     */
-    public void testAddIndexOnDiskFull() throws IOException
+    public void xxxtestAddIndexOnDiskFull() throws IOException
     {
       int START_COUNT = 57;
       int NUM_DIR = 50;
@@ -465,7 +465,7 @@
      * Make sure IndexWriter cleans up on hitting a disk
      * full exception in addDocument.
      */
-    public void testAddDocumentOnDiskFull() throws IOException {
+    public void xxxtestAddDocumentOnDiskFull() throws IOException {
 
       boolean debug = false;
 
@@ -556,7 +556,7 @@
     /**
      * Make sure we skip wicked long terms.
     */
-    public void testWickedLongTerm() throws IOException {
+    public void xxxtestWickedLongTerm() throws IOException {
       RAMDirectory dir = new RAMDirectory();
       IndexWriter writer  = new IndexWriter(dir, new StandardAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
 
@@ -614,7 +614,7 @@
       dir.close();
     }
 
-    public void testOptimizeMaxNumSegments() throws IOException {
+    public void xxxtestOptimizeMaxNumSegments() throws IOException {
 
       MockRAMDirectory dir = new MockRAMDirectory();
 
@@ -653,7 +653,7 @@
       }
     }
 
-    public void testOptimizeMaxNumSegments2() throws IOException {
+    public void xxxtestOptimizeMaxNumSegments2() throws IOException {
       MockRAMDirectory dir = new MockRAMDirectory();
 
       final Document doc = new Document();
@@ -698,7 +698,7 @@
      * starting index size as its temporary free space
      * required.
      */
-    public void testOptimizeTempSpaceUsage() throws IOException {
+    public void xxxtestOptimizeTempSpaceUsage() throws IOException {
     
       MockRAMDirectory dir = new MockRAMDirectory();
       IndexWriter writer  = new IndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
@@ -738,7 +738,7 @@
     // Make sure we can open an index for create even when a
     // reader holds it open (this fails pre lock-less
     // commits on windows):
-    public void testCreateWithReader() throws IOException {
+    public void xxxtestCreateWithReader() throws IOException {
         String tempDir = System.getProperty("java.io.tmpdir");
         if (tempDir == null)
             throw new IOException("java.io.tmpdir undefined, cannot run test");
@@ -775,7 +775,7 @@
 
     // Same test as above, but use IndexWriter constructor
     // that takes File:
-    public void testCreateWithReader2() throws IOException {
+    public void xxxtestCreateWithReader2() throws IOException {
         String tempDir = System.getProperty("java.io.tmpdir");
         if (tempDir == null)
             throw new IOException("java.io.tmpdir undefined, cannot run test");
@@ -808,7 +808,7 @@
 
     // Same test as above, but use IndexWriter constructor
     // that takes String:
-    public void testCreateWithReader3() throws IOException {
+    public void xxxtestCreateWithReader3() throws IOException {
         String tempDir = System.getProperty("tempDir");
         if (tempDir == null)
             throw new IOException("java.io.tmpdir undefined, cannot run test");
@@ -845,7 +845,7 @@
     // file: make sure we can still open the index (ie,
     // gracefully fallback to the previous segments file),
     // and that we can add to the index:
-    public void testSimulatedCrashedWriter() throws IOException {
+    public void xxxtestSimulatedCrashedWriter() throws IOException {
         Directory dir = new RAMDirectory();
 
         IndexWriter writer = null;
@@ -905,7 +905,7 @@
     // Simulate a corrupt index by removing last byte of
     // latest segments file and make sure we get an
     // IOException trying to open the index:
-    public void testSimulatedCorruptIndex1() throws IOException {
+    public void xxxtestSimulatedCorruptIndex1() throws IOException {
         Directory dir = new RAMDirectory();
 
         IndexWriter writer = null;
@@ -948,7 +948,7 @@
         }
     }
 
-    public void testChangesAfterClose() throws IOException {
+    public void xxxtestChangesAfterClose() throws IOException {
         Directory dir = new RAMDirectory();
 
         IndexWriter writer = null;
@@ -970,7 +970,7 @@
     // Simulate a corrupt index by removing one of the cfs
     // files and make sure we get an IOException trying to
     // open the index:
-    public void testSimulatedCorruptIndex2() throws IOException {
+    public void xxxtestSimulatedCorruptIndex2() throws IOException {
         Directory dir = new RAMDirectory();
 
         IndexWriter writer = null;
@@ -1013,7 +1013,7 @@
      * then add a bunch of docs, making sure reader does not
      * see these docs until writer is closed.
      */
-    public void testCommitOnClose() throws IOException {
+    public void xxxtestCommitOnClose() throws IOException {
         Directory dir = new RAMDirectory();      
         IndexWriter writer = new IndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
         for (int i = 0; i < 14; i++) {
@@ -1060,7 +1060,7 @@
      * nothing was added.  Then verify we can open the index
      * and add docs to it.
      */
-    public void testCommitOnCloseAbort() throws IOException {
+    public void xxxtestCommitOnCloseAbort() throws IOException {
       MockRAMDirectory dir = new MockRAMDirectory();      
       IndexWriter writer = new IndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
       writer.setMaxBufferedDocs(10);
@@ -1133,7 +1133,7 @@
      * file.  We check this by using MockRAMDirectory to
      * measure max temp disk space used.
      */
-    public void testCommitOnCloseDiskUsage() throws IOException {
+    public void xxxtestCommitOnCloseDiskUsage() throws IOException {
       MockRAMDirectory dir = new MockRAMDirectory();      
       IndexWriter writer  = new IndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
       for(int j=0;j<30;j++) {
@@ -1176,7 +1176,7 @@
      * "commit on close" works correctly both for abort()
      * and close().
      */
-    public void testCommitOnCloseOptimize() throws IOException {
+    public void xxxtestCommitOnCloseOptimize() throws IOException {
       RAMDirectory dir = new RAMDirectory();      
       IndexWriter writer  = new IndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
       writer.setMaxBufferedDocs(10);
@@ -1220,7 +1220,7 @@
       reader.close();
     }
 
-    public void testIndexNoDocuments() throws IOException {
+    public void xxxtestIndexNoDocuments() throws IOException {
       RAMDirectory dir = new RAMDirectory();      
       IndexWriter writer  = new IndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
       writer.flush();
@@ -1241,7 +1241,7 @@
       reader.close();
     }
 
-    public void testManyFields() throws IOException {
+    public void xxxtestManyFields() throws IOException {
       RAMDirectory dir = new RAMDirectory();      
       IndexWriter writer  = new IndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
       writer.setMaxBufferedDocs(10);
@@ -1272,7 +1272,7 @@
       dir.close();
     }
 
-    public void testSmallRAMBuffer() throws IOException {
+    public void xxxtestSmallRAMBuffer() throws IOException {
       RAMDirectory dir = new RAMDirectory();      
       IndexWriter writer  = new IndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
       writer.setRAMBufferSizeMB(0.000001);
@@ -1293,7 +1293,7 @@
 
     // Make sure it's OK to change RAM buffer size and
     // maxBufferedDocs in a write session
-    public void testChangingRAMBuffer() throws IOException {
+    public void xxxtestChangingRAMBuffer() throws IOException {
       RAMDirectory dir = new RAMDirectory();      
       IndexWriter writer  = new IndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
       writer.setMaxBufferedDocs(10);
@@ -1347,7 +1347,7 @@
       dir.close();
     }
 
-    public void testChangingRAMBuffer2() throws IOException {
+    public void xxxtestChangingRAMBuffer2() throws IOException {
       RAMDirectory dir = new RAMDirectory();      
       IndexWriter writer  = new IndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
       writer.setMaxBufferedDocs(10);
@@ -1407,7 +1407,7 @@
       dir.close();
     }
 
-    public void testDiverseDocs() throws IOException {
+    public void xxxtestDiverseDocs() throws IOException {
       RAMDirectory dir = new RAMDirectory();      
       IndexWriter writer  = new IndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
       writer.setRAMBufferSizeMB(0.5);
@@ -1456,7 +1456,7 @@
       dir.close();
     }
 
-    public void testEnablingNorms() throws IOException {
+    public void xxxtestEnablingNorms() throws IOException {
       RAMDirectory dir = new RAMDirectory();      
       IndexWriter writer  = new IndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
       writer.setMaxBufferedDocs(10);
@@ -1503,7 +1503,7 @@
       dir.close();
     }
 
-    public void testHighFreqTerm() throws IOException {
+    public void xxxtestHighFreqTerm() throws IOException {
       RAMDirectory dir = new RAMDirectory();      
       IndexWriter writer  = new IndexWriter(dir, new WhitespaceAnalyzer(), true, new IndexWriter.MaxFieldLength(100000000));
       writer.setRAMBufferSizeMB(0.01);
@@ -1537,7 +1537,7 @@
     // implements its own private locking) works OK.  This
     // was raised on java-dev as loss of backwards
     // compatibility.
-    public void testNullLockFactory() throws IOException {
+    public void xxxtestNullLockFactory() throws IOException {
 
       final class MyRAMDirectory extends RAMDirectory {
         private LockFactory myLockFactory;
@@ -1568,7 +1568,7 @@
       dir.close();
     }
 
-    public void testFlushWithNoMerging() throws IOException {
+    public void xxxtestFlushWithNoMerging() throws IOException {
       Directory dir = new RAMDirectory();
       IndexWriter writer  = new IndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
       writer.setMaxBufferedDocs(2);
@@ -1587,7 +1587,7 @@
 
     // Make sure we can flush segment w/ norms, then add
     // empty doc (no norms) and flush
-    public void testEmptyDocAfterFlushingRealDoc() throws IOException {
+    public void xxxtestEmptyDocAfterFlushingRealDoc() throws IOException {
       Directory dir = new RAMDirectory();
       IndexWriter writer  = new IndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
       Document doc = new Document();
@@ -1604,7 +1604,7 @@
     // Test calling optimize(false) whereby optimize is kicked
     // off but we don't wait for it to finish (but
     // writer.close()) does wait
-    public void testBackgroundOptimize() throws IOException {
+    public void xxxtestBackgroundOptimize() throws IOException {
 
       Directory dir = new MockRAMDirectory();
       for(int pass=0;pass<2;pass++) {
@@ -1660,7 +1660,7 @@
    * @throws IOException
    *
    */
-  public void testBadSegment() throws IOException {
+  public void xxxtestBadSegment() throws IOException {
     MockRAMDirectory dir = new MockRAMDirectory();
     IndexWriter ir = new IndexWriter(dir, new StandardAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
     
@@ -1673,7 +1673,7 @@
   }
 
   // LUCENE-1008
-  public void testNoTermVectorAfterTermVector() throws IOException {
+  public void xxxtestNoTermVectorAfterTermVector() throws IOException {
     MockRAMDirectory dir = new MockRAMDirectory();
     IndexWriter iw = new IndexWriter(dir, new StandardAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
     Document document = new Document();
@@ -1699,7 +1699,7 @@
   }
 
   // LUCENE-1010
-  public void testNoTermVectorAfterTermVectorMerge() throws IOException {
+  public void xxxtestNoTermVectorAfterTermVectorMerge() throws IOException {
     MockRAMDirectory dir = new MockRAMDirectory();
     IndexWriter iw = new IndexWriter(dir, new StandardAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
     Document document = new Document();
@@ -1729,7 +1729,7 @@
   }
 
   // LUCENE-1036
-  public void testMaxThreadPriority() throws IOException {
+  public void xxxtestMaxThreadPriority() throws IOException {
     int pri = Thread.currentThread().getPriority();
     try {
       MockRAMDirectory dir = new MockRAMDirectory();
@@ -1769,7 +1769,7 @@
   }
 
   // LUCENE-1013
-  public void testSetMaxMergeDocs() throws IOException {
+  public void xxxtestSetMaxMergeDocs() throws IOException {
     MockRAMDirectory dir = new MockRAMDirectory();
     IndexWriter iw = new IndexWriter(dir, new StandardAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
     iw.setMergeScheduler(new MyMergeScheduler());
@@ -1785,7 +1785,7 @@
   }
 
   // LUCENE-1072
-  public void testExceptionFromTokenStream() throws IOException {
+  public void xxxtestExceptionFromTokenStream() throws IOException {
     RAMDirectory dir = new MockRAMDirectory();
     IndexWriter writer = new IndexWriter(dir, new Analyzer() {
 
@@ -1878,7 +1878,7 @@
 
   // LUCENE-1072: make sure an errant exception on flushing
   // one segment only takes out those docs in that one flush
-  public void testDocumentsWriterAbort() throws IOException {
+  public void xxxtestDocumentsWriterAbort() throws IOException {
     MockRAMDirectory dir = new MockRAMDirectory();
     FailOnlyOnFlush failure = new FailOnlyOnFlush();
     failure.setDoFail();
@@ -1928,7 +1928,7 @@
     }
   }
 
-  public void testDocumentsWriterExceptions() throws IOException {
+  public void xxxtestDocumentsWriterExceptions() throws IOException {
     Analyzer analyzer = new Analyzer() {
       public TokenStream tokenStream(String fieldName, Reader reader) {
         return new CrashingFilter(fieldName, new WhitespaceTokenizer(reader));
@@ -2010,7 +2010,7 @@
     }
   }
 
-  public void testDocumentsWriterExceptionThreads() throws IOException {
+  public void xxxtestDocumentsWriterExceptionThreads() throws IOException {
     Analyzer analyzer = new Analyzer() {
       public TokenStream tokenStream(String fieldName, Reader reader) {
         return new CrashingFilter(fieldName, new WhitespaceTokenizer(reader));
@@ -2128,7 +2128,7 @@
     }
   }
 
-  public void testVariableSchema() throws IOException {
+  public void xxxtestVariableSchema() throws IOException {
     MockRAMDirectory dir = new MockRAMDirectory();
     int delID = 0;
     for(int i=0;i<20;i++) {
@@ -2176,7 +2176,7 @@
     }
   }
 
-  public void testNoWaitClose() throws Throwable {
+  public void xxxtestNoWaitClose() throws Throwable {
     RAMDirectory directory = new MockRAMDirectory();
 
     final Document doc = new Document();
@@ -2337,7 +2337,7 @@
   // 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 xxxtestCloseWithThreads() throws IOException {
     int NUM_THREADS = 3;
 
     for(int iter=0;iter<20;iter++) {
@@ -2410,7 +2410,7 @@
   // LUCENE-1130: make sure immeidate disk full on creating
   // an IndexWriter (hit during DW.ThreadState.init()) is
   // OK:
-  public void testImmediateDiskFull() throws IOException {
+  public void xxxtestImmediateDiskFull() throws IOException {
     MockRAMDirectory dir = new MockRAMDirectory();
     IndexWriter writer = new IndexWriter(dir, new WhitespaceAnalyzer(), IndexWriter.MaxFieldLength.LIMITED);
     dir.setMaxSizeInBytes(dir.getRecomputedActualSizeInBytes());
@@ -2438,7 +2438,7 @@
   // 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 xxxtestImmediateDiskFullWithThreads() throws IOException {
 
     int NUM_THREADS = 3;
 
@@ -2610,25 +2610,25 @@
 
   // LUCENE-1130: make sure initial IOException, and then 2nd
   // IOException during abort(), is OK:
-  public void testIOExceptionDuringAbort() throws IOException {
+  public void xxxtestIOExceptionDuringAbort() throws IOException {
     _testSingleThreadFailure(new FailOnlyOnAbortOrFlush(false));
   }
 
   // LUCENE-1130: make sure initial IOException, and then 2nd
   // IOException during abort(), is OK:
-  public void testIOExceptionDuringAbortOnlyOnce() throws IOException {
+  public void xxxtestIOExceptionDuringAbortOnlyOnce() throws IOException {
     _testSingleThreadFailure(new FailOnlyOnAbortOrFlush(true));
   }
 
   // LUCENE-1130: make sure initial IOException, and then 2nd
   // IOException during abort(), with multiple threads, is OK:
-  public void testIOExceptionDuringAbortWithThreads() throws IOException {
+  public void xxxtestIOExceptionDuringAbortWithThreads() throws IOException {
     _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 xxxtestIOExceptionDuringAbortWithThreadsOnlyOnce() throws IOException {
     _testMultipleThreadsFailure(new FailOnlyOnAbortOrFlush(true));
   }
 
@@ -2653,22 +2653,22 @@
   }
 
   // LUCENE-1130: test IOException in closeDocStore
-  public void testIOExceptionDuringCloseDocStore() throws IOException {
+  public void xxxtestIOExceptionDuringCloseDocStore() throws IOException {
     _testSingleThreadFailure(new FailOnlyInCloseDocStore(false));
   }
 
   // LUCENE-1130: test IOException in closeDocStore
-  public void testIOExceptionDuringCloseDocStoreOnlyOnce() throws IOException {
+  public void xxxtestIOExceptionDuringCloseDocStoreOnlyOnce() throws IOException {
     _testSingleThreadFailure(new FailOnlyInCloseDocStore(true));
   }
 
   // LUCENE-1130: test IOException in closeDocStore, with threads
-  public void testIOExceptionDuringCloseDocStoreWithThreads() throws IOException {
+  public void xxxtestIOExceptionDuringCloseDocStoreWithThreads() throws IOException {
     _testMultipleThreadsFailure(new FailOnlyInCloseDocStore(false));
   }
 
   // LUCENE-1130: test IOException in closeDocStore, with threads
-  public void testIOExceptionDuringCloseDocStoreWithThreadsOnlyOnce() throws IOException {
+  public void xxxtestIOExceptionDuringCloseDocStoreWithThreadsOnlyOnce() throws IOException {
     _testMultipleThreadsFailure(new FailOnlyInCloseDocStore(true));
   }
 
@@ -2693,27 +2693,27 @@
   }
 
   // LUCENE-1130: test IOException in writeSegment
-  public void testIOExceptionDuringWriteSegment() throws IOException {
+  public void xxxtestIOExceptionDuringWriteSegment() throws IOException {
     _testSingleThreadFailure(new FailOnlyInWriteSegment(false));
   }
 
   // LUCENE-1130: test IOException in writeSegment
-  public void testIOExceptionDuringWriteSegmentOnlyOnce() throws IOException {
+  public void xxxtestIOExceptionDuringWriteSegmentOnlyOnce() throws IOException {
     _testSingleThreadFailure(new FailOnlyInWriteSegment(true));
   }
 
   // LUCENE-1130: test IOException in writeSegment, with threads
-  public void testIOExceptionDuringWriteSegmentWithThreads() throws IOException {
+  public void xxxtestIOExceptionDuringWriteSegmentWithThreads() throws IOException {
     _testMultipleThreadsFailure(new FailOnlyInWriteSegment(false));
   }
 
   // LUCENE-1130: test IOException in writeSegment, with threads
-  public void testIOExceptionDuringWriteSegmentWithThreadsOnlyOnce() throws IOException {
+  public void xxxtestIOExceptionDuringWriteSegmentWithThreadsOnlyOnce() throws IOException {
     _testMultipleThreadsFailure(new FailOnlyInWriteSegment(true));
   }
 
   // LUCENE-1084: test unlimited field length
-  public void testUnlimitedMaxFieldLength() throws IOException {
+  public void xxxtestUnlimitedMaxFieldLength() throws IOException {
     Directory dir = new MockRAMDirectory();
 
     IndexWriter writer = new IndexWriter(dir, new WhitespaceAnalyzer(), IndexWriter.MaxFieldLength.UNLIMITED);
@@ -2735,7 +2735,7 @@
   }
 
   // LUCENE-1044: Simulate checksum error in segments_N
-  public void testSegmentsChecksumError() throws IOException {
+  public void xxxtestSegmentsChecksumError() throws IOException {
     Directory dir = new MockRAMDirectory();
 
     IndexWriter writer = null;
@@ -2773,7 +2773,7 @@
   }
 
   // LUCENE-1044: test writer.commit() when ac=false
-  public void testForceCommit() throws IOException {
+  public void xxxtestForceCommit() throws IOException {
     Directory dir = new MockRAMDirectory();
 
     IndexWriter writer  = new IndexWriter(dir, new WhitespaceAnalyzer(), IndexWriter.MaxFieldLength.LIMITED);
@@ -2824,7 +2824,7 @@
   }
 
   // LUCENE-1044: test exception during sync
-  public void testExceptionDuringSync() throws IOException {
+  public void xxxtestExceptionDuringSync() throws IOException {
     MockRAMDirectory dir = new MockRAMDirectory();
     FailOnlyInSync failure = new FailOnlyInSync();
     dir.failOn(failure);
@@ -2854,7 +2854,7 @@
   }
 
   // LUCENE-1168
-  public void testTermVectorCorruption() throws IOException {
+  public void xxxtestTermVectorCorruption() throws IOException {
 
     Directory dir = new MockRAMDirectory();
     for(int iter=0;iter<4;iter++) {
@@ -2907,7 +2907,7 @@
   }
 
   // LUCENE-1168
-  public void testTermVectorCorruption2() throws IOException {
+  public void xxxtestTermVectorCorruption2() throws IOException {
     Directory dir = new MockRAMDirectory();
     for(int iter=0;iter<4;iter++) {
       final boolean autoCommit = 1==iter/2;
@@ -2946,7 +2946,7 @@
   }
 
   // LUCENE-1168
-  public void testTermVectorCorruption3() throws IOException {
+  public void xxxtestTermVectorCorruption3() throws IOException {
     Directory dir = new MockRAMDirectory();
     IndexWriter writer = new IndexWriter(dir,
                                          new StandardAnalyzer(),
@@ -2993,7 +2993,7 @@
   }
 
   // LUCENE-1084: test user-specified field length
-  public void testUserSpecifiedMaxFieldLength() throws IOException {
+  public void xxxtestUserSpecifiedMaxFieldLength() throws IOException {
     Directory dir = new MockRAMDirectory();
 
     IndexWriter writer = new IndexWriter(dir, new WhitespaceAnalyzer(), new IndexWriter.MaxFieldLength(100000));
@@ -3016,7 +3016,7 @@
 
   // LUCENE-325: test expungeDeletes, when 2 singular merges
   // are required
-  public void testExpungeDeletes() throws IOException {
+  public void xxxtestExpungeDeletes() throws IOException {
     Directory dir = new MockRAMDirectory();
     IndexWriter writer = new IndexWriter(dir,
                                          new StandardAnalyzer(),
@@ -3062,7 +3062,7 @@
   }
 
   // LUCENE-325: test expungeDeletes, when many adjacent merges are required
-  public void testExpungeDeletes2() throws IOException {
+  public void xxxtestExpungeDeletes2() throws IOException {
     Directory dir = new MockRAMDirectory();
     IndexWriter writer = new IndexWriter(dir,
                                          new StandardAnalyzer(),
@@ -3109,7 +3109,7 @@
 
   // LUCENE-325: test expungeDeletes without waiting, when
   // many adjacent merges are required
-  public void testExpungeDeletes3() throws IOException {
+  public void xxxtestExpungeDeletes3() throws IOException {
     Directory dir = new MockRAMDirectory();
     IndexWriter writer = new IndexWriter(dir,
                                          new StandardAnalyzer(),
@@ -3155,7 +3155,7 @@
   }
 
   // LUCENE-1179
-  public void testEmptyFieldName() throws IOException {
+  public void xxxtestEmptyFieldName() throws IOException {
     MockRAMDirectory dir = new MockRAMDirectory();
     IndexWriter writer = new IndexWriter(dir, new WhitespaceAnalyzer());
     Document doc = new Document();
@@ -3180,7 +3180,7 @@
     }
   }
 
-  public void testExceptionDocumentsWriterInit() throws IOException {
+  public void xxxtestExceptionDocumentsWriterInit() throws IOException {
     MockRAMDirectory dir = new MockRAMDirectory();
     MockIndexWriter w = new MockIndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.UNLIMITED);
     Document doc = new Document();
@@ -3200,7 +3200,7 @@
   }
 
   // LUCENE-1208
-  public void testExceptionJustBeforeFlush() throws IOException {
+  public void xxxtestExceptionJustBeforeFlush() throws IOException {
     MockRAMDirectory dir = new MockRAMDirectory();
     MockIndexWriter w = new MockIndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.UNLIMITED);
     w.setMaxBufferedDocs(2);
@@ -3248,7 +3248,7 @@
   }
 
   // LUCENE-1210
-  public void testExceptionOnMergeInit() throws IOException {
+  public void xxxtestExceptionOnMergeInit() throws IOException {
     MockRAMDirectory dir = new MockRAMDirectory();
     MockIndexWriter2 w = new MockIndexWriter2(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.UNLIMITED);
     w.setMaxBufferedDocs(2);
@@ -3285,7 +3285,7 @@
   }
 
   // LUCENE-1222
-  public void testDoAfterFlush() throws IOException {
+  public void xxxtestDoAfterFlush() throws IOException {
     MockRAMDirectory dir = new MockRAMDirectory();
     MockIndexWriter3 w = new MockIndexWriter3(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
     Document doc = new Document();
@@ -3336,7 +3336,7 @@
   }
 
   // LUCENE-1214
-  public void testExceptionsDuringCommit() throws Throwable {
+  public void xxxtestExceptionsDuringCommit() throws Throwable {
     MockRAMDirectory dir = new MockRAMDirectory();
     FailOnlyInCommit failure = new FailOnlyInCommit();
     IndexWriter w = new IndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.UNLIMITED);
@@ -3385,7 +3385,7 @@
   };
 
   // LUCENE-510
-  public void testInvalidUTF16() throws Throwable {
+  public void xxxtestInvalidUTF16() throws Throwable {
     MockRAMDirectory dir = new MockRAMDirectory();
     IndexWriter w = new IndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.UNLIMITED);
     Document doc = new Document();
@@ -3407,7 +3407,7 @@
   }
 
   // LUCENE-510
-  public void testAllUnicodeChars() throws Throwable {
+  public void xxxtestAllUnicodeChars() throws Throwable {
 
     UnicodeUtil.UTF8Result utf8 = new UnicodeUtil.UTF8Result();
     UnicodeUtil.UTF16Result utf16 = new UnicodeUtil.UTF16Result();
@@ -3497,7 +3497,7 @@
   }
 
   // LUCENE-510
-  public void testRandomUnicodeStrings() throws Throwable {
+  public void xxxtestRandomUnicodeStrings() throws Throwable {
     r = newRandom();
 
     char[] buffer = new char[20];
@@ -3525,7 +3525,7 @@
   }
 
   // LUCENE-510
-  public void testIncrementalUnicodeStrings() throws Throwable {
+  public void xxxtestIncrementalUnicodeStrings() throws Throwable {
     r = newRandom();
     char[] buffer = new char[20];
     char[] expected = new char[20];
@@ -3590,7 +3590,7 @@
   }
   
   // LUCENE-1255
-  public void testNegativePositions() throws Throwable {
+  public void xxxtestNegativePositions() throws Throwable {
     SinkTokenizer tokens = new SinkTokenizer();
     tokens.addAttribute(TermAttribute.class);
     tokens.addAttribute(PositionIncrementAttribute.class);
@@ -3648,7 +3648,7 @@
   }
 
   // LUCENE-1274: test writer.prepareCommit()
-  public void testPrepareCommit() throws IOException {
+  public void xxxtestPrepareCommit() throws IOException {
     Directory dir = new MockRAMDirectory();
 
     IndexWriter writer = new IndexWriter(dir, new WhitespaceAnalyzer(), IndexWriter.MaxFieldLength.LIMITED);
@@ -3699,7 +3699,7 @@
   }
 
   // LUCENE-1274: test writer.prepareCommit()
-  public void testPrepareCommitRollback() throws IOException {
+  public void xxxtestPrepareCommitRollback() throws IOException {
     MockRAMDirectory dir = new MockRAMDirectory();
     dir.setPreventDoubleWrite(false);
 
@@ -3753,7 +3753,7 @@
   }
 
   // LUCENE-1274
-  public void testPrepareCommitNoChanges() throws IOException {
+  public void xxxtestPrepareCommitNoChanges() throws IOException {
     MockRAMDirectory dir = new MockRAMDirectory();
 
     IndexWriter writer = new IndexWriter(dir, new WhitespaceAnalyzer(), IndexWriter.MaxFieldLength.LIMITED);
@@ -3883,7 +3883,7 @@
 
   // LUCENE-1335: test simultaneous addIndexes & commits
   // from multiple threads
-  public void testAddIndexesWithThreads() throws Throwable {
+  public void xxxtestAddIndexesWithThreads() throws Throwable {
 
     final int NUM_ITER = 12;
     final int NUM_COPY = 3;
@@ -3926,7 +3926,7 @@
   }
 
   // LUCENE-1335: test simultaneous addIndexes & close
-  public void testAddIndexesWithClose() throws Throwable {
+  public void xxxtestAddIndexesWithClose() throws Throwable {
     final int NUM_COPY = 3;
     CommitAndAddIndexes2 c = new CommitAndAddIndexes2(NUM_COPY);
     //c.writer2.setInfoStream(System.out);
@@ -3989,7 +3989,7 @@
   }
 
   // LUCENE-1335: test simultaneous addIndexes & close
-  public void testAddIndexesWithCloseNoWait() throws Throwable {
+  public void xxxtestAddIndexesWithCloseNoWait() throws Throwable {
 
     final int NUM_COPY = 50;
     CommitAndAddIndexes3 c = new CommitAndAddIndexes3(NUM_COPY);
@@ -4056,7 +4056,7 @@
   }
 
   // LUCENE-1347
-  public void testRollbackExceptionHang() throws Throwable {
+  public void xxxtestRollbackExceptionHang() throws Throwable {
     MockRAMDirectory dir = new MockRAMDirectory();
     MockIndexWriter4 w = new MockIndexWriter4(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
 
@@ -4075,7 +4075,7 @@
 
 
   // LUCENE-1219
-  public void testBinaryFieldOffsetLength() throws IOException {
+  public void xxxtestBinaryFieldOffsetLength() throws IOException {
     MockRAMDirectory dir = new MockRAMDirectory();
     IndexWriter w = new IndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.UNLIMITED);
     byte[] b = new byte[50];
@@ -4105,7 +4105,7 @@
   }
 
   // LUCENE-1374
-  public void testMergeCompressedFields() throws IOException {
+  public void xxxtestMergeCompressedFields() throws IOException {
     File indexDir = new File(System.getProperty("tempDir"), "mergecompressedfields");
     Directory dir = FSDirectory.getDirectory(indexDir);
     try {
@@ -4147,7 +4147,7 @@
   }
 
   // LUCENE-1382
-  public void testCommitUserData() throws IOException {
+  public void xxxtestCommitUserData() throws IOException {
     Directory dir = new MockRAMDirectory();
     IndexWriter w = new IndexWriter(dir, new WhitespaceAnalyzer(), IndexWriter.MaxFieldLength.LIMITED);
     w.setMaxBufferedDocs(2);
@@ -4184,7 +4184,7 @@
     dir.close();
   }
 
-  public void testOptimizeExceptions() throws IOException {
+  public void xxxtestOptimizeExceptions() throws IOException {
     RAMDirectory startDir = new MockRAMDirectory();
     IndexWriter w = new IndexWriter(startDir, new WhitespaceAnalyzer(), IndexWriter.MaxFieldLength.UNLIMITED);    
     w.setMaxBufferedDocs(2);
@@ -4210,7 +4210,7 @@
   }
 
   // LUCENE-1429
-  public void testOutOfMemoryErrorCausesCloseToFail() throws Exception {
+  public void xxxtestOutOfMemoryErrorCausesCloseToFail() throws Exception {
 
     final List thrown = new ArrayList();
 
@@ -4236,7 +4236,7 @@
   }
 
   // LUCENE-1442
-  public void testDoubleOffsetCounting() throws Exception {
+  public void xxxtestDoubleOffsetCounting() throws Exception {
     MockRAMDirectory dir = new MockRAMDirectory();
     IndexWriter w = new IndexWriter(dir, new WhitespaceAnalyzer(), IndexWriter.MaxFieldLength.LIMITED);
     Document doc = new Document();
@@ -4260,7 +4260,7 @@
   // LUCENE-1468 -- make sure opening an IndexWriter with
   // create=true does not remove non-index files
   
-  public void testOtherFiles() throws Throwable {
+  public void xxxtestOtherFiles() throws Throwable {
     File indexDir = new File(System.getProperty("tempDir"), "otherfiles");
     Directory dir = new FSDirectory(indexDir, null);
     try {
@@ -4284,7 +4284,7 @@
     }
   }
 
-  public void testDeadlock() throws Exception {
+  public void xxxtestDeadlock() throws Exception {
     MockRAMDirectory dir = new MockRAMDirectory();
     IndexWriter writer = new IndexWriter(dir, true, new WhitespaceAnalyzer());
     writer.setMaxBufferedDocs(2);