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 2008/07/18 11:20:14 UTC

svn commit: r677865 [5/5] - in /lucene/java/trunk: ./ src/java/org/apache/lucene/index/ src/java/org/apache/lucene/store/ src/java/org/apache/lucene/util/ src/test/org/apache/lucene/ src/test/org/apache/lucene/index/ src/test/org/apache/lucene/search/

Modified: lucene/java/trunk/src/java/org/apache/lucene/store/RAMOutputStream.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/RAMOutputStream.java?rev=677865&r1=677864&r2=677865&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/RAMOutputStream.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/RAMOutputStream.java Fri Jul 18 02:20:12 2008
@@ -149,4 +149,9 @@
   public long getFilePointer() {
     return currentBufferIndex < 0 ? 0 : bufferStart + bufferPosition;
   }
+
+  /** Returns byte usage of all buffers. */
+  public long sizeInBytes() {
+    return file.numBuffers() * BUFFER_SIZE;
+  }
 }

Added: lucene/java/trunk/src/java/org/apache/lucene/util/ArrayUtil.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/ArrayUtil.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/ArrayUtil.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/ArrayUtil.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,109 @@
+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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+public final class ArrayUtil {
+
+  public static int getNextSize(int targetSize) {
+    /* This over-allocates proportional to the list size, making room
+     * for additional growth.  The over-allocation is mild, but is
+     * enough to give linear-time amortized behavior over a long
+     * sequence of appends() in the presence of a poorly-performing
+     * system realloc().
+     * The growth pattern is:  0, 4, 8, 16, 25, 35, 46, 58, 72, 88, ...
+     */
+    return (targetSize >> 3) + (targetSize < 9 ? 3 : 6) + targetSize;
+  }
+
+  public static int getShrinkSize(int currentSize, int targetSize) {
+    final int newSize = getNextSize(targetSize);
+    if (newSize < currentSize && currentSize > newSize*2)
+      return newSize;
+    else
+      return currentSize;
+  }
+
+  public static int[] grow(int[] array, int minSize) {
+    if (array.length < minSize) {
+      int[] newArray = new int[getNextSize(minSize)];
+      System.arraycopy(array, 0, newArray, 0, array.length);
+      return newArray;
+    } else
+      return array;
+  }
+
+  public static int[] grow(int[] array) {
+    return grow(array, 1+array.length);
+  }
+
+  public static int[] shrink(int[] array, int targetSize) {
+    final int newSize = getShrinkSize(array.length, targetSize);
+    if (newSize != array.length) {
+      int[] newArray = new int[newSize];
+      System.arraycopy(array, 0, newArray, 0, newSize);
+      return newArray;
+    } else
+      return array;
+  }
+
+  public static long[] grow(long[] array, int minSize) {
+    if (array.length < minSize) {
+      long[] newArray = new long[getNextSize(minSize)];
+      System.arraycopy(array, 0, newArray, 0, array.length);
+      return newArray;
+    } else
+      return array;
+  }
+
+  public static long[] grow(long[] array) {
+    return grow(array, 1+array.length);
+  }
+
+  public static long[] shrink(long[] array, int targetSize) {
+    final int newSize = getShrinkSize(array.length, targetSize);
+    if (newSize != array.length) {
+      long[] newArray = new long[newSize];
+      System.arraycopy(array, 0, newArray, 0, newSize);
+      return newArray;
+    } else
+      return array;
+  }
+
+  public static byte[] grow(byte[] array, int minSize) {
+    if (array.length < minSize) {
+      byte[] newArray = new byte[getNextSize(minSize)];
+      System.arraycopy(array, 0, newArray, 0, array.length);
+      return newArray;
+    } else
+      return array;
+  }
+
+  public static byte[] grow(byte[] array) {
+    return grow(array, 1+array.length);
+  }
+
+  public static byte[] shrink(byte[] array, int targetSize) {
+    final int newSize = getShrinkSize(array.length, targetSize);
+    if (newSize != array.length) {
+      byte[] newArray = new byte[newSize];
+      System.arraycopy(array, 0, newArray, 0, newSize);
+      return newArray;
+    } else
+      return array;
+  }
+}

Modified: lucene/java/trunk/src/test/org/apache/lucene/TestDemo.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/TestDemo.java?rev=677865&r1=677864&r2=677865&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/TestDemo.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/TestDemo.java Fri Jul 18 02:20:12 2008
@@ -32,6 +32,7 @@
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 
 /**
  * A very simple demo used in the API documentation (src/java/overview.html).
@@ -58,6 +59,8 @@
         Field.Index.TOKENIZED));
     iwriter.addDocument(doc);
     iwriter.close();
+
+    _TestUtil.checkIndex(directory);
     
     // Now search the index:
     IndexSearcher isearcher = new IndexSearcher(directory);

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=677865&r1=677864&r2=677865&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/TestSnapshotDeletionPolicy.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/TestSnapshotDeletionPolicy.java Fri Jul 18 02:20:12 2008
@@ -121,9 +121,8 @@
               try {
                 writer.addDocument(doc);
               } catch (Throwable t) {
-                RuntimeException re = new RuntimeException("addDocument failed");
-                re.initCause(t);
-                throw re;
+                t.printStackTrace(System.out);
+                fail("addDocument failed");
               }
             }
             try {

Modified: lucene/java/trunk/src/test/org/apache/lucene/index/TestDocumentWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/TestDocumentWriter.java?rev=677865&r1=677864&r2=677865&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/index/TestDocumentWriter.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/index/TestDocumentWriter.java Fri Jul 18 02:20:12 2008
@@ -27,6 +27,7 @@
 import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 
 import java.io.IOException;
 import java.io.Reader;
@@ -250,6 +251,8 @@
     writer.addDocument(doc);
     writer.close();
 
+    _TestUtil.checkIndex(ram);
+
     IndexReader reader = IndexReader.open(ram);
     // f1
     TermFreqVector tfv1 = reader.getTermFreqVector(0, "f1");

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=677865&r1=677864&r2=677865&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 Fri Jul 18 02:20:12 2008
@@ -553,7 +553,7 @@
       RAMDirectory dir = new RAMDirectory();
       IndexWriter writer  = new IndexWriter(dir, new StandardAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
 
-      char[] chars = new char[16383];
+      char[] chars = new char[DocumentsWriter.CHAR_BLOCK_SIZE-1];
       Arrays.fill(chars, 'x');
       Document doc = new Document();
       final String bigTerm = new String(chars);
@@ -1136,7 +1136,9 @@
       dir.resetMaxUsedSizeInBytes();
 
       long startDiskUsage = dir.getMaxUsedSizeInBytes();
-      writer  = new IndexWriter(dir, false, new WhitespaceAnalyzer(), false, IndexWriter.MaxFieldLength.LIMITED);
+      writer = new IndexWriter(dir, false, new WhitespaceAnalyzer(), false, IndexWriter.MaxFieldLength.LIMITED);
+      writer.setMaxBufferedDocs(10);
+      writer.setMergeScheduler(new SerialMergeScheduler());
       for(int j=0;j<1470;j++) {
         addDocWithIndex(writer, j);
       }
@@ -1144,6 +1146,9 @@
       dir.resetMaxUsedSizeInBytes();
       writer.optimize();
       writer.close();
+
+      IndexReader.open(dir).close();
+
       long endDiskUsage = dir.getMaxUsedSizeInBytes();
 
       // Ending index is 50X as large as starting index; due
@@ -1154,7 +1159,7 @@
       // System.out.println("start " + startDiskUsage + "; mid " + midDiskUsage + ";end " + endDiskUsage);
       assertTrue("writer used to much space while adding documents when autoCommit=false",     
                  midDiskUsage < 100*startDiskUsage);
-      assertTrue("writer used to much space after close when autoCommit=false",     
+      assertTrue("writer used to much space after close when autoCommit=false endDiskUsage=" + endDiskUsage + " startDiskUsage=" + startDiskUsage,
                  endDiskUsage < 100*startDiskUsage);
     }
 
@@ -1584,6 +1589,7 @@
       writer.flush();
       writer.addDocument(new Document());
       writer.close();
+      _TestUtil.checkIndex(dir);
       IndexReader reader = IndexReader.open(dir);
       assertEquals(2, reader.numDocs());
     }
@@ -1849,7 +1855,7 @@
         boolean sawAppend = false;
         boolean sawFlush = false;
         for (int i = 0; i < trace.length; i++) {
-          if ("org.apache.lucene.index.DocumentsWriter".equals(trace[i].getClassName()) && "appendPostings".equals(trace[i].getMethodName()))
+          if ("org.apache.lucene.index.FreqProxTermsWriter".equals(trace[i].getClassName()) && "appendPostings".equals(trace[i].getMethodName()))
             sawAppend = true;
           if ("doFlush".equals(trace[i].getMethodName()))
             sawFlush = true;
@@ -2287,6 +2293,7 @@
           writer.updateDocument(new Term("id", ""+(idUpto++)), doc);
           addCount++;
         } catch (IOException ioe) {
+          //System.out.println(Thread.currentThread().getName() + ": hit exc");
           //ioe.printStackTrace(System.out);
           if (ioe.getMessage().startsWith("fake disk full at") ||
               ioe.getMessage().equals("now failing on purpose")) {
@@ -2484,6 +2491,8 @@
               "flushDocument".equals(trace[i].getMethodName())) {
             if (onlyOnce)
               doFail = false;
+            //System.out.println(Thread.currentThread().getName() + ": now fail");
+            //new Throwable().printStackTrace(System.out);
             throw new IOException("now failing on purpose");
           }
         }
@@ -2663,10 +2672,9 @@
       if (doFail) {
         StackTraceElement[] trace = new Exception().getStackTrace();
         for (int i = 0; i < trace.length; i++) {
-          if ("writeSegment".equals(trace[i].getMethodName())) {
+          if ("flush".equals(trace[i].getMethodName()) && "org.apache.lucene.index.DocFieldProcessor".equals(trace[i].getClassName())) {
             if (onlyOnce)
               doFail = false;
-            // new RuntimeException().printStackTrace(System.out);
             throw new IOException("now failing on purpose");
           }
         }

Added: lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (added)
+++ lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,215 @@
+package org.apache.lucene.index;
+
+/**
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+import java.util.Random;
+import java.io.IOException;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
+import org.apache.lucene.store.MockRAMDirectory;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.WhitespaceAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+
+public class TestIndexWriterExceptions extends LuceneTestCase {
+
+  final private static boolean DEBUG = false;
+
+  private class IndexerThread extends Thread {
+
+    IndexWriter writer;
+
+    final Random r = new java.util.Random(47);
+    Throwable failure;
+
+    public IndexerThread(int i, IndexWriter writer) {
+      setName("Indexer " + i);
+      this.writer = writer;
+    }
+
+    public void run() {
+
+      final Document doc = new Document();
+
+      doc.add(new Field("content1", "aaa bbb ccc ddd", Field.Store.YES, Field.Index.TOKENIZED));
+      doc.add(new Field("content6", "aaa bbb ccc ddd", Field.Store.NO, Field.Index.TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+      doc.add(new Field("content2", "aaa bbb ccc ddd", Field.Store.YES, Field.Index.UN_TOKENIZED));
+      doc.add(new Field("content3", "aaa bbb ccc ddd", Field.Store.YES, Field.Index.NO));
+
+      doc.add(new Field("content4", "aaa bbb ccc ddd", Field.Store.NO, Field.Index.TOKENIZED));
+      doc.add(new Field("content5", "aaa bbb ccc ddd", Field.Store.NO, Field.Index.UN_TOKENIZED));
+
+      doc.add(new Field("content7", "aaa bbb ccc ddd", Field.Store.NO, Field.Index.UN_TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+
+      final Field idField = new Field("id", "", Field.Store.YES, Field.Index.UN_TOKENIZED);
+      doc.add(idField);
+
+      final long stopTime = System.currentTimeMillis() + 3000;
+
+      while(System.currentTimeMillis() < stopTime) {
+        doFail.set(this);
+        final String id = ""+r.nextInt(50);
+        idField.setValue(id);
+        Term idTerm = new Term("id", id);
+        try {
+          writer.updateDocument(idTerm, doc);
+        } catch (RuntimeException re) {
+          if (DEBUG) {
+            System.out.println("EXC: ");
+            re.printStackTrace(System.out);
+          }
+          try {
+            _TestUtil.checkIndex(writer.getDirectory());
+          } catch (IOException ioe) {
+            System.out.println(Thread.currentThread().getName() + ": unexpected exception1");
+            ioe.printStackTrace(System.out);
+            failure = ioe;
+            break;
+          }
+        } catch (Throwable t) {
+          System.out.println(Thread.currentThread().getName() + ": unexpected exception2");
+          t.printStackTrace(System.out);
+          failure = t;
+          break;
+        }
+
+        doFail.set(null);
+
+        // After a possible exception (above) I should be able
+        // to add a new document without hitting an
+        // exception:
+        try {
+          writer.updateDocument(idTerm, doc);
+        } catch (Throwable t) {
+          System.out.println(Thread.currentThread().getName() + ": unexpected exception3");
+          t.printStackTrace(System.out);
+          failure = t;
+          break;
+        }
+      }
+    }
+  }
+
+  ThreadLocal doFail = new ThreadLocal();
+
+  public class MockIndexWriter extends IndexWriter {
+    Random r = new java.util.Random(17);
+
+    public MockIndexWriter(Directory dir, Analyzer a, boolean create, MaxFieldLength mfl) throws IOException {
+      super(dir, false, a, create, mfl);
+    }
+
+    boolean testPoint(String name) {
+      if (doFail.get() != null && !name.equals("startDoFlush") && r.nextInt(20) == 17) {
+        if (DEBUG) {
+          System.out.println(Thread.currentThread().getName() + ": NOW FAIL: " + name);
+          //new Throwable().printStackTrace(System.out);
+        }
+        throw new RuntimeException(Thread.currentThread().getName() + ": intentionally failing at " + name);
+      }
+      return true;
+    }
+  }
+
+  public void testRandomExceptions() throws Throwable {
+    MockRAMDirectory dir = new MockRAMDirectory();
+
+    MockIndexWriter writer  = new MockIndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
+    ((ConcurrentMergeScheduler) writer.getMergeScheduler()).setSuppressExceptions();
+    //writer.setMaxBufferedDocs(10);
+    writer.setRAMBufferSizeMB(0.1);
+
+    if (DEBUG)
+      writer.setInfoStream(System.out);
+
+    IndexerThread thread = new IndexerThread(0, writer);
+    thread.run();
+    if (thread.failure != null) {
+      thread.failure.printStackTrace(System.out);
+      fail("thread " + thread.getName() + ": hit unexpected failure");
+    }
+
+    writer.commit();
+
+    try {
+      writer.close();
+    } catch (Throwable t) {
+      System.out.println("exception during close:");
+      t.printStackTrace(System.out);
+      writer.rollback();
+    }
+
+    // Confirm that when doc hits exception partway through tokenization, it's deleted:
+    IndexReader r2 = IndexReader.open(dir);
+    final int count = r2.docFreq(new Term("content4", "aaa"));
+    final int count2 = r2.docFreq(new Term("content4", "ddd"));
+    assertEquals(count, count2);
+    r2.close();
+
+    _TestUtil.checkIndex(dir);
+  }
+
+  public void testRandomExceptionsThreads() throws Throwable {
+
+    MockRAMDirectory dir = new MockRAMDirectory();
+    MockIndexWriter writer  = new MockIndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
+    ((ConcurrentMergeScheduler) writer.getMergeScheduler()).setSuppressExceptions();
+    //writer.setMaxBufferedDocs(10);
+    writer.setRAMBufferSizeMB(0.2);
+
+    if (DEBUG)
+      writer.setInfoStream(System.out);
+
+    final int NUM_THREADS = 4;
+
+    final IndexerThread[] threads = new IndexerThread[NUM_THREADS];
+    for(int i=0;i<NUM_THREADS;i++) {
+      threads[i] = new IndexerThread(i, writer);
+      threads[i].start();
+    }
+
+    for(int i=0;i<NUM_THREADS;i++)
+      threads[i].join();
+
+    for(int i=0;i<NUM_THREADS;i++)
+      if (threads[i].failure != null)
+        fail("thread " + threads[i].getName() + ": hit unexpected failure");
+
+    writer.commit();
+
+    try {
+      writer.close();
+    } catch (Throwable t) {
+      System.out.println("exception during close:");
+      t.printStackTrace(System.out);
+      writer.rollback();
+    }
+
+    // Confirm that when doc hits exception partway through tokenization, it's deleted:
+    IndexReader r2 = IndexReader.open(dir);
+    final int count = r2.docFreq(new Term("content4", "aaa"));
+    final int count2 = r2.docFreq(new Term("content4", "ddd"));
+    assertEquals(count, count2);
+    r2.close();
+
+    _TestUtil.checkIndex(dir);
+  }
+}

Propchange: lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/java/trunk/src/test/org/apache/lucene/index/TestStressIndexing2.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/TestStressIndexing2.java?rev=677865&r1=677864&r2=677865&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/index/TestStressIndexing2.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/index/TestStressIndexing2.java Fri Jul 18 02:20:12 2008
@@ -19,11 +19,10 @@
 import org.apache.lucene.analysis.*;
 
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 import org.apache.lucene.search.TermQuery;
 
 import java.util.*;
-import java.io.ByteArrayOutputStream;
-import java.io.PrintStream;
 import java.io.IOException;
 
 import junit.framework.TestCase;
@@ -53,7 +52,7 @@
     }
   }
 
-  public void testRandom() throws Exception {
+  public void testRandom() throws Throwable {
     Directory dir1 = new MockRAMDirectory();
     // dir1 = FSDirectory.getDirectory("foofoofoo");
     Directory dir2 = new MockRAMDirectory();
@@ -68,17 +67,7 @@
     verifyEquals(dir1, dir2, "id");
   }
 
-  private void checkIndex(Directory dir) throws IOException {
-    ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
-    CheckIndex.out = new PrintStream(bos);
-    if (!CheckIndex.check(dir, false, null)) {
-      System.out.println("CheckIndex failed");
-      System.out.println(bos.toString());
-      fail("CheckIndex failed");
-    }
-  }
-
-  public void testMultiConfig() throws Exception {
+  public void testMultiConfig() throws Throwable {
     // test lots of smaller different params together
     for (int i=0; i<100; i++) {  // increase iterations for better testing
       sameFieldOrder=r.nextBoolean();
@@ -157,7 +146,7 @@
       }
     }
 
-    checkIndex(dir);
+    _TestUtil.checkIndex(dir);
 
     return docs;
   }
@@ -187,7 +176,7 @@
     w.close();
   }
 
-  public static void verifyEquals(Directory dir1, Directory dir2, String idField) throws IOException {
+  public static void verifyEquals(Directory dir1, Directory dir2, String idField) throws Throwable {
     IndexReader r1 = IndexReader.open(dir1);
     IndexReader r2 = IndexReader.open(dir2);
     verifyEquals(r1, r2, idField);
@@ -196,7 +185,7 @@
   }
 
 
-  public static void verifyEquals(IndexReader r1, IndexReader r2, String idField) throws IOException {
+  public static void verifyEquals(IndexReader r1, IndexReader r2, String idField) throws Throwable {
     assertEquals(r1.numDocs(), r2.numDocs());
     boolean hasDeletes = !(r1.maxDoc()==r2.maxDoc() && r1.numDocs()==r1.maxDoc());
 
@@ -236,7 +225,7 @@
       try {
         // verify term vectors are equivalent        
         verifyEquals(r1.getTermFreqVectors(id1), r2.getTermFreqVectors(id2));
-      } catch (java.lang.Error e) {
+      } catch (Throwable e) {
         System.out.println("FAILED id=" + term + " id1=" + id1 + " id2=" + id2);
         TermFreqVector[] tv1 = r1.getTermFreqVectors(id1);
         System.out.println("  d1=" + tv1);
@@ -367,6 +356,8 @@
     for(int i=0;i<d1.length;i++) {
       TermFreqVector v1 = d1[i];
       TermFreqVector v2 = d2[i];
+      if (v1 == null || v2 == null)
+        System.out.println("v1=" + v1 + " v2=" + v2 + " i=" + i + " of " + d1.length);
       assertEquals(v1.size(), v2.size());
       int numTerms = v1.size();
       String[] terms1 = v1.getTerms();
@@ -572,7 +563,7 @@
             indexDoc();
           }
         }
-      } catch (Exception e) {
+      } catch (Throwable e) {
         e.printStackTrace();
         TestCase.fail(e.toString());
       }

Modified: lucene/java/trunk/src/test/org/apache/lucene/search/TestTermVectors.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/search/TestTermVectors.java?rev=677865&r1=677864&r2=677865&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/search/TestTermVectors.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/search/TestTermVectors.java Fri Jul 18 02:20:12 2008
@@ -93,7 +93,7 @@
   
   public void testTermVectorsFieldOrder() throws IOException {
     Directory dir = new MockRAMDirectory();
-    IndexWriter writer = new IndexWriter(dir, new SimpleAnalyzer(), true);
+    IndexWriter writer = new IndexWriter(dir, new SimpleAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
     Document doc = new Document();
     doc.add(new Field("c", "some content here", Field.Store.YES, Field.Index.TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
     doc.add(new Field("a", "some content here", Field.Store.YES, Field.Index.TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));