You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2011/01/05 21:25:44 UTC

svn commit: r1055622 [7/14] - in /lucene/dev/branches/docvalues: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/ dev-tools/idea/.idea/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/ dev-tools/idea/lucene/contrib/ dev-tools/idea/lucene/contrib/...

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java Wed Jan  5 20:25:17 2011
@@ -24,16 +24,15 @@ import org.apache.lucene.document.Numeri
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.SlowMultiReaderWrapper;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util._TestUtil;
-
-import org.junit.Test;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import static org.junit.Assert.*;
+import org.junit.Test;
 
 public class TestNumericRangeQuery64 extends LuceneTestCase {
   // distance of entries
@@ -183,13 +182,13 @@ public class TestNumericRangeQuery64 ext
   @Test
   public void testInverseRange() throws Exception {
     NumericRangeFilter<Long> f = NumericRangeFilter.newLongRange("field8", 8, 1000L, -1000L, true, true);
-    assertSame("A inverse range should return the EMPTY_DOCIDSET instance", DocIdSet.EMPTY_DOCIDSET, f.getDocIdSet(searcher.getIndexReader()));
+    assertSame("A inverse range should return the EMPTY_DOCIDSET instance", DocIdSet.EMPTY_DOCIDSET, f.getDocIdSet(new SlowMultiReaderWrapper(searcher.getIndexReader())));
     f = NumericRangeFilter.newLongRange("field8", 8, Long.MAX_VALUE, null, false, false);
     assertSame("A exclusive range starting with Long.MAX_VALUE should return the EMPTY_DOCIDSET instance",
-      DocIdSet.EMPTY_DOCIDSET, f.getDocIdSet(searcher.getIndexReader()));
+               DocIdSet.EMPTY_DOCIDSET, f.getDocIdSet(new SlowMultiReaderWrapper(searcher.getIndexReader())));
     f = NumericRangeFilter.newLongRange("field8", 8, null, Long.MIN_VALUE, false, false);
     assertSame("A exclusive range ending with Long.MIN_VALUE should return the EMPTY_DOCIDSET instance",
-      DocIdSet.EMPTY_DOCIDSET, f.getDocIdSet(searcher.getIndexReader()));
+               DocIdSet.EMPTY_DOCIDSET, f.getDocIdSet(new SlowMultiReaderWrapper(searcher.getIndexReader())));
   }
   
   @Test

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestPhraseQuery.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestPhraseQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestPhraseQuery.java Wed Jan  5 20:25:17 2011
@@ -352,7 +352,7 @@ public class TestPhraseQuery extends Luc
     IndexReader reader = writer.getReader();
     writer.close();
 
-    Searcher searcher = new IndexSearcher(reader);
+    IndexSearcher searcher = new IndexSearcher(reader);
     PhraseQuery query = new PhraseQuery();
     query.add(new Term("field", "firstname"));
     query.add(new Term("field", "lastname"));

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestQueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestQueryWrapperFilter.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestQueryWrapperFilter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestQueryWrapperFilter.java Wed Jan  5 20:25:17 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Field.Index;
 import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.index.IndexReader;

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestRegexpRandom.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestRegexpRandom.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestRegexpRandom.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestRegexpRandom.java Wed Jan  5 20:25:17 2011
@@ -38,7 +38,7 @@ import org.apache.lucene.util._TestUtil;
  * and validates the correct number of hits are returned.
  */
 public class TestRegexpRandom extends LuceneTestCase {
-  private Searcher searcher;
+  private IndexSearcher searcher;
   private IndexReader reader;
   private Directory dir;
   

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java Wed Jan  5 20:25:17 2011
@@ -49,7 +49,7 @@ import org.apache.lucene.util.automaton.
  * Generates random regexps, and validates against a simple impl.
  */
 public class TestRegexpRandom2 extends LuceneTestCase {
-  private IndexSearcher searcher;
+  protected IndexSearcher searcher;
   private IndexReader reader;
   private Directory dir;
   
@@ -146,7 +146,7 @@ public class TestRegexpRandom2 extends L
   /** check that the # of hits is the same as from a very
    * simple regexpquery implementation.
    */
-  private void assertSame(String regexp) throws IOException {   
+  protected void assertSame(String regexp) throws IOException {   
     RegexpQuery smart = new RegexpQuery(new Term("field", regexp), RegExp.NONE);
     DumbRegexpQuery dumb = new DumbRegexpQuery(new Term("field", regexp), RegExp.NONE);
     

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestSimilarity.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestSimilarity.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestSimilarity.java Wed Jan  5 20:25:17 2011
@@ -44,7 +44,7 @@ public class TestSimilarity extends Luce
     @Override public float sloppyFreq(int distance) { return 2.0f; }
     @Override public float idf(int docFreq, int numDocs) { return 1.0f; }
     @Override public float coord(int overlap, int maxOverlap) { return 1.0f; }
-    @Override public IDFExplanation idfExplain(Collection<Term> terms, Searcher searcher) throws IOException {
+    @Override public IDFExplanation idfExplain(Collection<Term> terms, IndexSearcher searcher) throws IOException {
       return new IDFExplanation() {
         @Override
         public float getIdf() {
@@ -75,7 +75,7 @@ public class TestSimilarity extends Luce
     IndexReader reader = writer.getReader();
     writer.close();
 
-    Searcher searcher = new IndexSearcher(reader);
+    IndexSearcher searcher = new IndexSearcher(reader);
     searcher.setSimilarity(new SimpleSimilarity());
 
     Term a = new Term("field", "a");

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestSimpleExplanations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestSimpleExplanations.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestSimpleExplanations.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestSimpleExplanations.java Wed Jan  5 20:25:17 2011
@@ -17,18 +17,6 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.queryParser.QueryParser;
-import org.apache.lucene.search.spans.SpanNearQuery;
-import org.apache.lucene.search.spans.SpanQuery;
-import org.apache.lucene.search.spans.SpanTermQuery;
-import org.apache.lucene.store.Directory;
-
-
 /**
  * TestExplanations subclass focusing on basic query types
  */
@@ -301,73 +289,4 @@ public class TestSimpleExplanations exte
     qtest(q, new int[] { 0,3 });
     
   }
-  
-  
-  public void testTermQueryMultiSearcherExplain() throws Exception {
-    // creating two directories for indices
-    Directory indexStoreA = newDirectory();
-    Directory indexStoreB = newDirectory();
-
-    Document lDoc = new Document();
-    lDoc.add(newField("handle", "1 2", Field.Store.YES, Field.Index.ANALYZED));
-    Document lDoc2 = new Document();
-    lDoc2.add(newField("handle", "1 2", Field.Store.YES, Field.Index.ANALYZED));
-    Document lDoc3 = new Document();
-    lDoc3.add(newField("handle", "1 2", Field.Store.YES, Field.Index.ANALYZED));
-
-    IndexWriter writerA = new IndexWriter(indexStoreA, newIndexWriterConfig(
-        TEST_VERSION_CURRENT, new MockAnalyzer()));
-    IndexWriter writerB = new IndexWriter(indexStoreB, newIndexWriterConfig(
-        TEST_VERSION_CURRENT, new MockAnalyzer()));
-
-    writerA.addDocument(lDoc);
-    writerA.addDocument(lDoc2);
-    writerA.optimize();
-    writerA.close();
-
-    writerB.addDocument(lDoc3);
-    writerB.close();
-
-    QueryParser parser = new QueryParser(TEST_VERSION_CURRENT, "fulltext", new MockAnalyzer());
-    Query query = parser.parse("handle:1");
-
-    Searcher[] searchers = new Searcher[2];
-    searchers[0] = new IndexSearcher(indexStoreB, true);
-    searchers[1] = new IndexSearcher(indexStoreA, true);
-    Searcher mSearcher = new MultiSearcher(searchers);
-    ScoreDoc[] hits = mSearcher.search(query, null, 1000).scoreDocs;
-
-    assertEquals(3, hits.length);
-
-    Explanation explain = mSearcher.explain(query, hits[0].doc);
-    String exp = explain.toString(0);
-    assertTrue(exp, exp.indexOf("maxDocs=3") > -1);
-    assertTrue(exp, exp.indexOf("docFreq=3") > -1);
-    
-    query = parser.parse("handle:\"1 2\"");
-    hits = mSearcher.search(query, null, 1000).scoreDocs;
-
-    assertEquals(3, hits.length);
-
-    explain = mSearcher.explain(query, hits[0].doc);
-    exp = explain.toString(0);
-    assertTrue(exp, exp.indexOf("1=3") > -1);
-    assertTrue(exp, exp.indexOf("2=3") > -1);
-    
-    query = new SpanNearQuery(new SpanQuery[] {
-        new SpanTermQuery(new Term("handle", "1")),
-        new SpanTermQuery(new Term("handle", "2")) }, 0, true);
-    hits = mSearcher.search(query, null, 1000).scoreDocs;
-
-    assertEquals(3, hits.length);
-
-    explain = mSearcher.explain(query, hits[0].doc);
-    exp = explain.toString(0);
-    assertTrue(exp, exp.indexOf("1=3") > -1);
-    assertTrue(exp, exp.indexOf("2=3") > -1);
-    mSearcher.close();
-    indexStoreA.close();
-    indexStoreB.close();
-  }
-  
 }

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestSort.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestSort.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestSort.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestSort.java Wed Jan  5 20:25:17 2011
@@ -25,7 +25,6 @@ import java.util.BitSet;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Locale;
-import java.util.concurrent.Executor;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
@@ -37,10 +36,9 @@ import org.apache.lucene.index.CorruptIn
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.LogMergePolicy;
+import org.apache.lucene.index.MultiReader;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.queryParser.ParseException;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.FieldValueHitQueue.Entry;
@@ -51,8 +49,9 @@ import org.apache.lucene.search.cache.Fl
 import org.apache.lucene.search.cache.IntValuesCreator;
 import org.apache.lucene.search.cache.LongValuesCreator;
 import org.apache.lucene.search.cache.ShortValuesCreator;
-import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.LockObtainFailedException;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.DocIdBitSet;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
@@ -458,7 +457,7 @@ public class TestSort extends LuceneTest
 
   // test sorts when there's nothing in the index
   public void testEmptyIndex() throws Exception {
-    Searcher empty = getEmptyIndex();
+    IndexSearcher empty = getEmptyIndex();
 
     sort = new Sort();
     assertMatches (empty, queryX, sort, "");
@@ -601,23 +600,9 @@ public class TestSort extends LuceneTest
         new SortField ("float", SortField.FLOAT, true) );
     assertMatches (full, queryG, sort, "ZYXW");
 
-    // Do the same for a MultiSearcher
-    Searcher multiSearcher=new MultiSearcher (full);
-
-    sort.setSort (new SortField ("int", SortField.INT),
-                                new SortField ("string", SortField.STRING),
-        new SortField ("float", SortField.FLOAT) );
-    assertMatches (multiSearcher, queryG, sort, "ZWXY");
-
-    sort.setSort (new SortField ("int", SortField.INT),
-                                new SortField ("string", SortField.STRING),
-        new SortField ("float", SortField.FLOAT, true) );
-    assertMatches (multiSearcher, queryG, sort, "ZYXW");
-    // Don't close the multiSearcher. it would close the full searcher too!
-
     // Do the same for a ParallelMultiSearcher
     ExecutorService exec = Executors.newFixedThreadPool(_TestUtil.nextInt(random, 2, 8));
-    Searcher parallelSearcher=new ParallelMultiSearcher (exec, full);
+    IndexSearcher parallelSearcher=new IndexSearcher (full.getIndexReader(), exec);
 
     sort.setSort (new SortField ("int", SortField.INT),
                                 new SortField ("string", SortField.STRING),
@@ -629,6 +614,7 @@ public class TestSort extends LuceneTest
         new SortField ("float", SortField.FLOAT, true) );
     assertMatches (parallelSearcher, queryG, sort, "ZYXW");
     parallelSearcher.close();
+    exec.shutdown();
     exec.awaitTermination(1000, TimeUnit.MILLISECONDS);
   }
 
@@ -674,117 +660,19 @@ public class TestSort extends LuceneTest
     assertMatches (full, queryX, sort, "EACGI");
   }
     
-    // Test the MultiSearcher's ability to preserve locale-sensitive ordering
-    // by wrapping it around a single searcher
-  public void testInternationalMultiSearcherSort() throws Exception {
-    Searcher multiSearcher = new MultiSearcher (full);
-    
-    sort.setSort (new SortField ("i18n", new Locale("sv", "se")));
-    assertMatches (multiSearcher, queryY, sort, "BJDFH");
-    
-    sort.setSort (new SortField ("i18n", Locale.US));
-    assertMatches (multiSearcher, queryY, sort, oStrokeFirst ? "BFJHD" : "BFJDH");
-    
-    sort.setSort (new SortField ("i18n", new Locale("da", "dk")));
-    assertMatches (multiSearcher, queryY, sort, "BJDHF");
-  } 
-
-  // test a variety of sorts using more than one searcher
-  public void testMultiSort() throws Exception {
-    MultiSearcher searcher = new MultiSearcher (searchX, searchY);
-    runMultiSorts(searcher, false);
-  }
-
   // test a variety of sorts using a parallel multisearcher
   public void testParallelMultiSort() throws Exception {
     ExecutorService exec = Executors.newFixedThreadPool(_TestUtil.nextInt(random, 2, 8));
-    Searcher searcher = new ParallelMultiSearcher (exec, searchX, searchY);
+    IndexSearcher searcher = new IndexSearcher(
+                                  new MultiReader(
+                                       new IndexReader[] {searchX.getIndexReader(),
+                                                          searchY.getIndexReader()}), exec);
     runMultiSorts(searcher, false);
     searcher.close();
+    exec.shutdown();
     exec.awaitTermination(1000, TimeUnit.MILLISECONDS);
   }
 
-  // test that the relevancy scores are the same even if
-  // hits are sorted
-  public void testNormalizedScores() throws Exception {
-
-    // capture relevancy scores
-    HashMap<String,Float> scoresX = getScores (full.search (queryX, null, 1000).scoreDocs, full);
-    HashMap<String,Float> scoresY = getScores (full.search (queryY, null, 1000).scoreDocs, full);
-    HashMap<String,Float> scoresA = getScores (full.search (queryA, null, 1000).scoreDocs, full);
-
-    // we'll test searching locally, remote and multi
-    
-    MultiSearcher multi  = new MultiSearcher (searchX, searchY);
-
-    // change sorting and make sure relevancy stays the same
-
-    sort = new Sort();
-    assertSameValues (scoresX, getScores (full.search (queryX, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresX, getScores (multi.search (queryX, null, 1000, sort).scoreDocs, multi));
-    assertSameValues (scoresY, getScores (full.search (queryY, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresY, getScores (multi.search (queryY, null, 1000, sort).scoreDocs, multi));
-    assertSameValues (scoresA, getScores (full.search (queryA, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresA, getScores (multi.search (queryA, null, 1000, sort).scoreDocs, multi));
-
-    sort.setSort(SortField.FIELD_DOC);
-    assertSameValues (scoresX, getScores (full.search (queryX, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresX, getScores (multi.search (queryX, null, 1000, sort).scoreDocs, multi));
-    assertSameValues (scoresY, getScores (full.search (queryY, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresY, getScores (multi.search (queryY, null, 1000, sort).scoreDocs, multi));
-    assertSameValues (scoresA, getScores (full.search (queryA, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresA, getScores (multi.search (queryA, null, 1000, sort).scoreDocs, multi));
-
-    sort.setSort (new SortField("int", SortField.INT));
-    assertSameValues (scoresX, getScores (full.search (queryX, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresX, getScores (multi.search (queryX, null, 1000, sort).scoreDocs, multi));
-    assertSameValues (scoresY, getScores (full.search (queryY, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresY, getScores (multi.search (queryY, null, 1000, sort).scoreDocs, multi));
-    assertSameValues (scoresA, getScores (full.search (queryA, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresA, getScores (multi.search (queryA, null, 1000, sort).scoreDocs, multi));
-
-    sort.setSort (new SortField("float", SortField.FLOAT));
-    assertSameValues (scoresX, getScores (full.search (queryX, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresX, getScores (multi.search (queryX, null, 1000, sort).scoreDocs, multi));
-    assertSameValues (scoresY, getScores (full.search (queryY, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresY, getScores (multi.search (queryY, null, 1000, sort).scoreDocs, multi));
-    assertSameValues (scoresA, getScores (full.search (queryA, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresA, getScores (multi.search (queryA, null, 1000, sort).scoreDocs, multi));
-
-    sort.setSort (new SortField("string", SortField.STRING));
-    assertSameValues (scoresX, getScores (full.search (queryX, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresX, getScores (multi.search (queryX, null, 1000, sort).scoreDocs, multi));
-    assertSameValues (scoresY, getScores (full.search (queryY, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresY, getScores (multi.search (queryY, null, 1000, sort).scoreDocs, multi));
-    assertSameValues (scoresA, getScores (full.search (queryA, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresA, getScores (multi.search (queryA, null, 1000, sort).scoreDocs, multi));
-
-    sort.setSort (new SortField("int", SortField.INT),new SortField("float", SortField.FLOAT));
-    assertSameValues (scoresX, getScores (full.search (queryX, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresX, getScores (multi.search (queryX, null, 1000, sort).scoreDocs, multi));
-    assertSameValues (scoresY, getScores (full.search (queryY, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresY, getScores (multi.search (queryY, null, 1000, sort).scoreDocs, multi));
-    assertSameValues (scoresA, getScores (full.search (queryA, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresA, getScores (multi.search (queryA, null, 1000, sort).scoreDocs, multi));
-
-    sort.setSort (new SortField ("int", SortField.INT, true), new SortField (null, SortField.DOC, true) );
-    assertSameValues (scoresX, getScores (full.search (queryX, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresX, getScores (multi.search (queryX, null, 1000, sort).scoreDocs, multi));
-    assertSameValues (scoresY, getScores (full.search (queryY, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresY, getScores (multi.search (queryY, null, 1000, sort).scoreDocs, multi));
-    assertSameValues (scoresA, getScores (full.search (queryA, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresA, getScores (multi.search (queryA, null, 1000, sort).scoreDocs, multi));
-
-    sort.setSort (new SortField("int", SortField.INT),new SortField("string", SortField.STRING));
-    assertSameValues (scoresX, getScores (full.search (queryX, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresX, getScores (multi.search (queryX, null, 1000, sort).scoreDocs, multi));
-    assertSameValues (scoresY, getScores (full.search (queryY, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresY, getScores (multi.search (queryY, null, 1000, sort).scoreDocs, multi));
-    assertSameValues (scoresA, getScores (full.search (queryA, null, 1000, sort).scoreDocs, full));
-    assertSameValues (scoresA, getScores (multi.search (queryA, null, 1000, sort).scoreDocs, multi));
-
-  }
-
   public void testTopDocsScores() throws Exception {
 
     // There was previously a bug in FieldSortedHitQueue.maxscore when only a single
@@ -1026,7 +914,7 @@ public class TestSort extends LuceneTest
   }
   
   // runs a variety of sorts useful for multisearchers
-  private void runMultiSorts(Searcher multi, boolean isFull) throws Exception {
+  private void runMultiSorts(IndexSearcher multi, boolean isFull) throws Exception {
     sort.setSort(SortField.FIELD_DOC);
     String expected = isFull ? "ABCDEFGHIJ" : "ACEGIBDFHJ";
     assertMatches(multi, queryA, sort, expected);
@@ -1103,12 +991,12 @@ public class TestSort extends LuceneTest
 
   }
 
-  private void assertMatches(Searcher searcher, Query query, Sort sort, String expectedResult) throws IOException {
+  private void assertMatches(IndexSearcher searcher, Query query, Sort sort, String expectedResult) throws IOException {
     assertMatches( null, searcher, query, sort, expectedResult );
   }
 
   // make sure the documents returned by the search match the expected list
-  private void assertMatches(String msg, Searcher searcher, Query query, Sort sort,
+  private void assertMatches(String msg, IndexSearcher searcher, Query query, Sort sort,
       String expectedResult) throws IOException {
     //ScoreDoc[] result = searcher.search (query, null, 1000, sort).scoreDocs;
     TopDocs hits = searcher.search (query, null, Math.max(1, expectedResult.length()), sort);
@@ -1126,7 +1014,7 @@ public class TestSort extends LuceneTest
     assertEquals (msg, expectedResult, buff.toString());
   }
 
-  private HashMap<String,Float> getScores (ScoreDoc[] hits, Searcher searcher)
+  private HashMap<String,Float> getScores (ScoreDoc[] hits, IndexSearcher searcher)
   throws IOException {
     HashMap<String,Float> scoreMap = new HashMap<String,Float>();
     int n = hits.length;

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestSubScorerFreqs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestSubScorerFreqs.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestSubScorerFreqs.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestSubScorerFreqs.java Wed Jan  5 20:25:17 2011
@@ -31,8 +31,6 @@ import org.junit.Test;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
-import static org.junit.Assert.*;
-
 public class TestSubScorerFreqs extends LuceneTestCase {
 
   private static Directory dir;

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestTermRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestTermRangeFilter.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestTermRangeFilter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestTermRangeFilter.java Wed Jan  5 20:25:17 2011
@@ -29,8 +29,6 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.store.Directory;
 import org.junit.Test;
 
-import static org.junit.Assert.*;
-
 /**
  * A basic 'positive' Unit test class for the TermRangeFilter class.
  * 

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestTermRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestTermRangeQuery.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestTermRangeQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestTermRangeQuery.java Wed Jan  5 20:25:17 2011
@@ -134,7 +134,7 @@ public class TestTermRangeQuery extends 
     searcher.close();
   }
   
-  private void checkBooleanTerms(Searcher searcher, TermRangeQuery query, String... terms) throws IOException {
+  private void checkBooleanTerms(IndexSearcher searcher, TermRangeQuery query, String... terms) throws IOException {
     query.setRewriteMethod(new MultiTermQuery.TopTermsScoringBooleanQueryRewrite(50));
     final BooleanQuery bq = (BooleanQuery) searcher.rewrite(query);
     final Set<String> allowedTerms = asSet(terms);

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java Wed Jan  5 20:25:17 2011
@@ -30,12 +30,14 @@ import org.apache.lucene.search.TimeLimi
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.ThreadInterruptedException;
+import org.junit.Ignore;
 
 /**
  * Tests the {@link TimeLimitingCollector}.  This test checks (1) search
  * correctness (regardless of timeout), (2) expected timeout behavior,
  * and (3) a sanity test with multiple searching threads.
  */
+@Ignore("broken: see https://issues.apache.org/jira/browse/LUCENE-2822")
 public class TestTimeLimitingCollector extends LuceneTestCase {
   private static final int SLOW_DOWN = 3;
   private static final long TIME_ALLOWED = 17 * SLOW_DOWN; // so searches can find about 17 docs.
@@ -48,7 +50,7 @@ public class TestTimeLimitingCollector e
   private static final int N_DOCS = 3000;
   private static final int N_THREADS = 50;
 
-  private Searcher searcher;
+  private IndexSearcher searcher;
   private Directory directory;
   private IndexReader reader;
 

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestWildcardRandom.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestWildcardRandom.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestWildcardRandom.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestWildcardRandom.java Wed Jan  5 20:25:17 2011
@@ -38,7 +38,7 @@ import org.apache.lucene.util._TestUtil;
  * and validates the correct number of hits are returned.
  */
 public class TestWildcardRandom extends LuceneTestCase {
-  private Searcher searcher;
+  private IndexSearcher searcher;
   private IndexReader reader;
   private Directory dir;
   

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/function/TestCustomScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/function/TestCustomScoreQuery.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/function/TestCustomScoreQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/function/TestCustomScoreQuery.java Wed Jan  5 20:25:17 2011
@@ -21,9 +21,6 @@ import org.apache.lucene.queryParser.Que
 import org.apache.lucene.queryParser.ParseException;
 import org.apache.lucene.search.*;
 import org.junit.Test;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
@@ -332,7 +329,7 @@ public class TestCustomScoreQuery extend
     }
   }
 
-  private void logResult(String msg, Searcher s, Query q, int doc, float score1) throws IOException {
+  private void logResult(String msg, IndexSearcher s, Query q, int doc, float score1) throws IOException {
     log(msg+" "+score1);
     log("Explain by: "+q);
     log(s.explain(q,doc));

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/function/TestDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/function/TestDocValues.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/function/TestDocValues.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/function/TestDocValues.java Wed Jan  5 20:25:17 2011
@@ -19,8 +19,6 @@ package org.apache.lucene.search.functio
 
 import org.apache.lucene.util.LuceneTestCase;
 import org.junit.Test;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 /**
  * DocValues TestCase  

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/function/TestFieldScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/function/TestFieldScoreQuery.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/function/TestFieldScoreQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/function/TestFieldScoreQuery.java Wed Jan  5 20:25:17 2011
@@ -26,7 +26,6 @@ import org.apache.lucene.search.QueryUti
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TopDocs;
 import org.junit.Test;
-import static org.junit.Assert.*;
 
 /**
  * Test FieldScoreQuery search.

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/function/TestOrdValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/function/TestOrdValues.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/function/TestOrdValues.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/function/TestOrdValues.java Wed Jan  5 20:25:17 2011
@@ -20,7 +20,6 @@ package org.apache.lucene.search.functio
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.*;
-import static org.junit.Assert.*;
 import org.junit.Test;
 
 /**

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java Wed Jan  5 20:25:17 2011
@@ -32,10 +32,9 @@ import org.apache.lucene.index.RandomInd
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.DefaultSimilarity;
 import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.QueryUtils;
 import org.apache.lucene.search.ScoreDoc;
-import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.spans.SpanQuery;
 import org.apache.lucene.search.spans.SpanNearQuery;
@@ -325,7 +324,7 @@ public class TestPayloadNearQuery extend
       return 1.0f;
     }
     // idf used for phrase queries
-    @Override public IDFExplanation idfExplain(Collection<Term> terms, Searcher searcher) throws IOException {
+    @Override public IDFExplanation idfExplain(Collection<Term> terms, IndexSearcher searcher) throws IOException {
       return new IDFExplanation() {
         @Override
         public float getIdf() {

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/spans/TestBasics.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/spans/TestBasics.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/spans/TestBasics.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/spans/TestBasics.java Wed Jan  5 20:25:17 2011
@@ -48,8 +48,6 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import static org.junit.Assert.*;
-
 /**
  * Tests basic search capabilities.
  *

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/spans/TestSpanMultiTermQueryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/spans/TestSpanMultiTermQueryWrapper.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/spans/TestSpanMultiTermQueryWrapper.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/spans/TestSpanMultiTermQueryWrapper.java Wed Jan  5 20:25:17 2011
@@ -24,7 +24,6 @@ import org.apache.lucene.index.RandomInd
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.FuzzyQuery;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Searcher;
 import org.apache.lucene.search.WildcardQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
@@ -35,7 +34,7 @@ import org.apache.lucene.util.LuceneTest
 public class TestSpanMultiTermQueryWrapper extends LuceneTestCase {
   private Directory directory;
   private IndexReader reader;
-  private Searcher searcher;
+  private IndexSearcher searcher;
   
   @Override
   public void setUp() throws Exception {

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/spans/TestSpans.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/spans/TestSpans.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/spans/TestSpans.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/spans/TestSpans.java Wed Jan  5 20:25:17 2011
@@ -18,14 +18,13 @@ package org.apache.lucene.search.spans;
  */
 
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.CheckHits;
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.search.DefaultSimilarity;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.index.IndexWriter;
@@ -416,7 +415,7 @@ public class TestSpans extends LuceneTes
                               slop,
                               ordered) {
       @Override
-      public Similarity getSimilarity(Searcher s) {
+      public Similarity getSimilarity(IndexSearcher s) {
         return sim;
       }
     };
@@ -439,7 +438,7 @@ public class TestSpans extends LuceneTes
   }
 
   // LUCENE-1404
-  private int hitCount(Searcher searcher, String word) throws Throwable {
+  private int hitCount(IndexSearcher searcher, String word) throws Throwable {
     return searcher.search(new TermQuery(new Term("text", word)), 10).totalHits;
   }
 

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/spans/TestSpansAdvanced.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/spans/TestSpansAdvanced.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/spans/TestSpansAdvanced.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/spans/TestSpansAdvanced.java Wed Jan  5 20:25:17 2011
@@ -134,7 +134,7 @@ public class TestSpansAdvanced extends L
    * 
    * @throws IOException
    */
-  protected static void assertHits(Searcher s, Query query,
+  protected static void assertHits(IndexSearcher s, Query query,
       final String description, final String[] expectedIds,
       final float[] expectedScores) throws IOException {
     QueryUtils.check(random, query, s);

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/store/MockDirectoryWrapper.java Wed Jan  5 20:25:17 2011
@@ -18,18 +18,20 @@ package org.apache.lucene.store;
  */
 
 import java.io.Closeable;
-import java.io.IOException;
 import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.IdentityHashMap;
 import java.util.Iterator;
-import java.util.Random;
 import java.util.Map;
-import java.util.HashMap;
-import java.util.HashSet;
+import java.util.Random;
 import java.util.Set;
-import java.util.ArrayList;
+
+import org.apache.lucene.util.LuceneTestCase;
 
 /**
  * This is a Directory Wrapper that adds methods
@@ -220,6 +222,10 @@ public class MockDirectoryWrapper extend
     if (randomIOExceptionRate > 0.0) {
       int number = Math.abs(randomState.nextInt() % 1000);
       if (number < randomIOExceptionRate*1000) {
+        if (LuceneTestCase.VERBOSE) {
+          System.out.println(Thread.currentThread().getName() + ": MockDirectoryWrapper: now throw random exception");
+          new Throwable().printStackTrace(System.out);
+        }
         throw new IOException("a random IOException");
       }
     }

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java Wed Jan  5 20:25:17 2011
@@ -32,7 +32,6 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.LogMergePolicy;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.search.IndexSearcher;

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/store/TestCopyBytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/store/TestCopyBytes.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/store/TestCopyBytes.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/store/TestCopyBytes.java Wed Jan  5 20:25:17 2011
@@ -22,7 +22,6 @@ import org.apache.lucene.util.LuceneTest
 import org.apache.lucene.util._TestUtil;
 
 import org.junit.Test;
-import static org.junit.Assert.*;
 
 public class TestCopyBytes extends LuceneTestCase {
 

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java Wed Jan  5 20:25:17 2011
@@ -26,7 +26,6 @@ import org.apache.lucene.index.IndexFile
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.LogMergePolicy;
 import org.apache.lucene.index.TestIndexWriterReader;
 import org.apache.lucene.util.LuceneTestCase;
 

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/LuceneJUnitResultFormatter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/LuceneJUnitResultFormatter.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/LuceneJUnitResultFormatter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/LuceneJUnitResultFormatter.java Wed Jan  5 20:25:17 2011
@@ -35,6 +35,7 @@ import org.apache.tools.ant.taskdefs.opt
 import org.apache.tools.ant.taskdefs.optional.junit.JUnitTestRunner;
 import org.apache.tools.ant.util.FileUtils;
 import org.apache.tools.ant.util.StringUtils;
+import org.junit.Ignore;
 
 /**
  * Just like BriefJUnitResultFormatter "brief" bundled with ant,
@@ -151,6 +152,18 @@ public class LuceneJUnitResultFormatter 
       .append(StringUtils.LINE_SEP);
     }
     
+    // HACK: junit gives us no way to do this in LuceneTestCase
+    try {
+      Class<?> clazz = Class.forName(suite.getName());
+      Ignore ignore = clazz.getAnnotation(Ignore.class);
+      if (ignore != null) {
+        if (systemError == null) systemError = "";
+        systemError += "NOTE: Ignoring test class '" + clazz.getSimpleName() + "': " 
+                    + ignore.value() + StringUtils.LINE_SEP;
+      }
+    } catch (ClassNotFoundException e) { /* no problem */ }
+    // END HACK
+    
     if (systemError != null && systemError.length() > 0) {
       append("------------- Standard Error -----------------")
       .append(StringUtils.LINE_SEP)

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java Wed Jan  5 20:25:17 2011
@@ -42,6 +42,7 @@ import org.apache.lucene.index.codecs.Co
 import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockCodec;
 import org.apache.lucene.index.codecs.mockintblock.MockVariableIntBlockCodec;
 import org.apache.lucene.index.codecs.mocksep.MockSepCodec;
+import org.apache.lucene.index.codecs.mockrandom.MockRandomCodec;
 import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
 import org.apache.lucene.index.codecs.preflexrw.PreFlexRWCodec;
 import org.apache.lucene.index.codecs.pulsing.PulsingCodec;
@@ -199,7 +200,7 @@ public abstract class LuceneTestCase ext
   
   private static Map<MockDirectoryWrapper,StackTraceElement[]> stores;
   
-  private static final String[] TEST_CODECS = new String[] {"MockSep", "MockFixedIntBlock", "MockVariableIntBlock"};
+  private static final String[] TEST_CODECS = new String[] {"MockSep", "MockFixedIntBlock", "MockVariableIntBlock", "MockRandom"};
 
   private static void swapCodec(Codec c, CodecProvider cp) {
     Codec prior = null;
@@ -252,6 +253,7 @@ public abstract class LuceneTestCase ext
     swapCodec(new MockFixedIntBlockCodec(codecHasParam && "MockFixedIntBlock".equals(codec) ? codecParam : _TestUtil.nextInt(random, 1, 2000)), cp);
     // baseBlockSize cannot be over 127:
     swapCodec(new MockVariableIntBlockCodec(codecHasParam && "MockVariableIntBlock".equals(codec) ? codecParam : _TestUtil.nextInt(random, 1, 127)), cp);
+    swapCodec(new MockRandomCodec(random), cp);
 
     return cp.lookup(codec);
   }
@@ -268,9 +270,9 @@ public abstract class LuceneTestCase ext
     cp.unregister(cp.lookup("MockSep"));
     cp.unregister(cp.lookup("MockFixedIntBlock"));
     cp.unregister(cp.lookup("MockVariableIntBlock"));
+    cp.unregister(cp.lookup("MockRandom"));
     swapCodec(new PulsingCodec(1), cp);
     cp.setDefaultFieldCodec(savedDefaultCodec);
-
   }
 
   // randomly picks from core and test codecs
@@ -522,6 +524,13 @@ public abstract class LuceneTestCase ext
   // jvm-wide list of 'rogue threads' we found, so they only get reported once.
   private final static IdentityHashMap<Thread,Boolean> rogueThreads = new IdentityHashMap<Thread,Boolean>();
   
+  static {
+    // just a hack for things like eclipse test-runner threads
+    for (Thread t : Thread.getAllStackTraces().keySet()) {
+      rogueThreads.put(t, true);
+    }
+  }
+  
   /**
    * Looks for leftover running threads, trying to kill them off,
    * so they don't fail future tests.
@@ -545,9 +554,7 @@ public abstract class LuceneTestCase ext
           
         if (t.isAlive() && 
             !rogueThreads.containsKey(t) && 
-            t != Thread.currentThread() &&
-            // TODO: TimeLimitingCollector starts a thread statically.... WTF?!
-            !t.getName().equals("TimeLimitedCollector timer thread")) {
+            t != Thread.currentThread()) {
           System.err.println("WARNING: " + context  + " left thread running: " + t);
           rogueThreads.put(t, true);
           rogueCount++;

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/TestCharacterUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/TestCharacterUtils.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/TestCharacterUtils.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/TestCharacterUtils.java Wed Jan  5 20:25:17 2011
@@ -17,9 +17,6 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
-import static org.junit.Assert.*;
-import static org.apache.lucene.util.LuceneTestCase.TEST_VERSION_CURRENT;
-
 import java.io.IOException;
 import java.io.Reader;
 import java.io.StringReader;

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/TestRecyclingByteBlockAllocator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/TestRecyclingByteBlockAllocator.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/TestRecyclingByteBlockAllocator.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/TestRecyclingByteBlockAllocator.java Wed Jan  5 20:25:17 2011
@@ -2,11 +2,8 @@ package org.apache.lucene.util;
 
 import java.util.ArrayList;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.ReentrantLock;
-
 import org.junit.Before;
 import org.junit.Test;
 

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/TestSetOnce.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/TestSetOnce.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/TestSetOnce.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/TestSetOnce.java Wed Jan  5 20:25:17 2011
@@ -17,8 +17,6 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
-import static org.junit.Assert.*;
-
 import java.util.Random;
 
 import org.apache.lucene.util.SetOnce.AlreadySetException;

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/automaton/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/automaton/fst/TestFSTs.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/automaton/fst/TestFSTs.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/automaton/fst/TestFSTs.java Wed Jan  5 20:25:17 2011
@@ -20,13 +20,13 @@ package org.apache.lucene.util.automaton
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileInputStream;
-import java.io.InputStreamReader;
 import java.io.IOException;
+import java.io.InputStreamReader;
 import java.io.PrintStream;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
@@ -122,8 +122,10 @@ public class TestFSTs extends LuceneTest
   }
 
   public void testBasicFSA() throws IOException {
-    String[] strings = new String[] {"station", "commotion", "elation", "elastic", "plastic", "stop", "ftop", "ftation"};
+    String[] strings = new String[] {"station", "commotion", "elation", "elastic", "plastic", "stop", "ftop", "ftation", "stat"};
+    String[] strings2 = new String[] {"station", "commotion", "elation", "elastic", "plastic", "stop", "ftop", "ftation"};
     IntsRef[] terms = new IntsRef[strings.length];
+    IntsRef[] terms2 = new IntsRef[strings2.length];
     for(int inputMode=0;inputMode<2;inputMode++) {
       if (VERBOSE) {
         System.out.println("TEST: inputMode=" + inputModeToString(inputMode));
@@ -132,6 +134,10 @@ public class TestFSTs extends LuceneTest
       for(int idx=0;idx<strings.length;idx++) {
         terms[idx] = toIntsRef(strings[idx], inputMode);
       }
+      for(int idx=0;idx<strings2.length;idx++) {
+        terms2[idx] = toIntsRef(strings2[idx], inputMode);
+      }
+      Arrays.sort(terms2);
 
       doTest(inputMode, terms);
     
@@ -141,8 +147,8 @@ public class TestFSTs extends LuceneTest
       {
         final Outputs<Object> outputs = NoOutputs.getSingleton();
         final Object NO_OUTPUT = outputs.getNoOutput();      
-        final List<FSTTester.InputOutput<Object>> pairs = new ArrayList<FSTTester.InputOutput<Object>>(terms.length);
-        for(IntsRef term : terms) {
+        final List<FSTTester.InputOutput<Object>> pairs = new ArrayList<FSTTester.InputOutput<Object>>(terms2.length);
+        for(IntsRef term : terms2) {
           pairs.add(new FSTTester.InputOutput<Object>(term, NO_OUTPUT));
         }
         FST<Object> fst = new FSTTester<Object>(random, dir, inputMode, pairs, outputs).doTest(0, 0);
@@ -154,9 +160,9 @@ public class TestFSTs extends LuceneTest
       // FST ord pos int
       {
         final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
-        final List<FSTTester.InputOutput<Long>> pairs = new ArrayList<FSTTester.InputOutput<Long>>(terms.length);
-        for(int idx=0;idx<terms.length;idx++) {
-          pairs.add(new FSTTester.InputOutput<Long>(terms[idx], outputs.get(idx)));
+        final List<FSTTester.InputOutput<Long>> pairs = new ArrayList<FSTTester.InputOutput<Long>>(terms2.length);
+        for(int idx=0;idx<terms2.length;idx++) {
+          pairs.add(new FSTTester.InputOutput<Long>(terms2[idx], outputs.get(idx)));
         }
         final FST<Long> fst = new FSTTester<Long>(random, dir, inputMode, pairs, outputs).doTest(0, 0);
         assertNotNull(fst);
@@ -168,10 +174,10 @@ public class TestFSTs extends LuceneTest
       {
         final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
         final BytesRef NO_OUTPUT = outputs.getNoOutput();      
-        final List<FSTTester.InputOutput<BytesRef>> pairs = new ArrayList<FSTTester.InputOutput<BytesRef>>(terms.length);
-        for(int idx=0;idx<terms.length;idx++) {
+        final List<FSTTester.InputOutput<BytesRef>> pairs = new ArrayList<FSTTester.InputOutput<BytesRef>>(terms2.length);
+        for(int idx=0;idx<terms2.length;idx++) {
           final BytesRef output = random.nextInt(30) == 17 ? NO_OUTPUT : new BytesRef(Integer.toString(idx));
-          pairs.add(new FSTTester.InputOutput<BytesRef>(terms[idx], output));
+          pairs.add(new FSTTester.InputOutput<BytesRef>(terms2[idx], output));
         }
         final FST<BytesRef> fst = new FSTTester<BytesRef>(random, dir, inputMode, pairs, outputs).doTest(0, 0);
         assertNotNull(fst);
@@ -322,19 +328,6 @@ public class TestFSTs extends LuceneTest
       }
     }
 
-    private String getRandomString() {
-      final String term;
-      if (random.nextBoolean()) {
-        term = _TestUtil.randomRealisticUnicodeString(random);
-      } else {
-        // we want to mix in limited-alphabet symbols so
-        // we get more sharing of the nodes given how few
-        // terms we are testing...
-        term = simpleRandomString(random);
-      }
-      return term;
-    }
-
     public void doTest() throws IOException {
       // no pruning
       doTest(0, 0);
@@ -346,156 +339,83 @@ public class TestFSTs extends LuceneTest
       doTest(0, _TestUtil.nextInt(random, 1, 1+pairs.size()));
     }
 
-    // NOTE: only copies the stuff this test needs!!
-    private FST.Arc<T> copyArc(FST.Arc<T> arc) {
-      final FST.Arc<T> copy = new FST.Arc<T>();
-      copy.label = arc.label;
-      copy.target = arc.target;
-      copy.output = arc.output;
-      copy.nextFinalOutput = arc.nextFinalOutput;
-      return arc;
-    }
-
     // runs the term, returning the output, or null if term
-    // isn't accepted.  if stopNode is non-null it must be
-    // length 2 int array; stopNode[0] will be the last
-    // matching node (-1 if the term is accepted)
-    // and stopNode[1] will be the length of the
-    // term prefix that matches
-    private T run(FST<T> fst, IntsRef term, int[] stopNode) throws IOException {
-      if (term.length == 0) {
-        final T output = fst.getEmptyOutput();
-        if (stopNode != null) {
-          stopNode[1] = 0;
-          if (output != null) {
-            // accepted
-            stopNode[0] = -1;
-          } else {
-            stopNode[0] = fst.getStartNode();
-          }
+    // isn't accepted.  if prefixLength is non-null it must be
+    // length 1 int array; prefixLength[0] is set to the length
+    // of the term prefix that matches
+    private T run(FST<T> fst, IntsRef term, int[] prefixLength) throws IOException {
+      assert prefixLength == null || prefixLength.length == 1;
+      final FST.Arc<T> arc = fst.getFirstArc(new FST.Arc<T>());
+      final T NO_OUTPUT = fst.outputs.getNoOutput();
+      T output = NO_OUTPUT;
+
+      for(int i=0;i<=term.length;i++) {
+        final int label;
+        if (i == term.length) {
+          label = FST.END_LABEL;
+        } else {
+          label = term.ints[term.offset+i];
         }
-        return output;
-      }
-
-      final FST.Arc<T> arc = new FST.Arc<T>();
-      int node = fst.getStartNode();
-      int lastNode = -1;
-      T output = fst.outputs.getNoOutput();
-      //System.out.println("match?");
-      for(int i=0;i<term.length;i++) {
-        //System.out.println("  int=" + term.ints[i]);
-        if (!fst.hasArcs(node)) {
-          //System.out.println("    no arcs!");
-          // hit end node before term's end
-          if (stopNode != null) {
-            stopNode[0] = lastNode;
-            stopNode[1] = i-1;
+        //System.out.println("   loop i=" + i + " label=" + label + " output=" + fst.outputs.outputToString(output) + " curArc: target=" + arc.target + " isFinal?=" + arc.isFinal());
+        if (fst.findTargetArc(label, arc, arc) == null) {
+          if (prefixLength != null) {
+            prefixLength[0] = i;
             return output;
           } else {
             return null;
           }
         }
-
-        if (fst.findArc(node, term.ints[term.offset + i], arc) != null) {
-          node = arc.target;
-          //System.out.println("    match final?=" + arc.isFinal());
-          if (arc.output != fst.outputs.getNoOutput()) {
-            output = fst.outputs.add(output, arc.output);
-          }
-        } else if (stopNode != null) {
-          stopNode[0] = node;
-          stopNode[1] = i;
-          return output;
-        } else {
-          //System.out.println("    no match");
-          return null;
-        }
-
-        lastNode = node;
-      }
-
-      if (!arc.isFinal()) {
-        // hit term's end before end node
-        if (stopNode != null) {
-          stopNode[0] = node;
-          stopNode[1] = term.length;
-          return output;
-        } else {
-          return null;
-        }
+        output = fst.outputs.add(output, arc.output);
       }
 
-      if (arc.nextFinalOutput != fst.outputs.getNoOutput()) {
-        output = fst.outputs.add(output, arc.nextFinalOutput);
+      if (prefixLength != null) {
+        prefixLength[0] = term.length;
       }
 
-      if (stopNode != null) {
-        stopNode[0] = -1;
-        stopNode[1] = term.length;
-      }
       return output;
     }
 
     private T randomAcceptedWord(FST<T> fst, IntsRef in) throws IOException {
-      int node = fst.getStartNode();
+      FST.Arc<T> arc = fst.getFirstArc(new FST.Arc<T>());
 
-      if (fst.noNodes()) {
-        // degenerate FST: only accepts the empty string
-        assertTrue(fst.getEmptyOutput() != null);
-        in.length = 0;
-        return fst.getEmptyOutput();
-      }
       final List<FST.Arc<T>> arcs = new ArrayList<FST.Arc<T>>();
       in.length = 0;
       in.offset = 0;
-      T output = fst.outputs.getNoOutput();
-      //System.out.println("get random");
+      final T NO_OUTPUT = fst.outputs.getNoOutput();
+      T output = NO_OUTPUT;
+
       while(true) {
         // read all arcs:
-        //System.out.println("  n=" + node);
-        int arcAddress = node;
-        FST.Arc<T> arc = new FST.Arc<T>();
-        fst.readFirstArc(arcAddress, arc);
-        arcs.add(copyArc(arc));
+        fst.readFirstTargetArc(arc, arc);
+        arcs.add(new FST.Arc<T>().copyFrom(arc));
         while(!arc.isLast()) {
           fst.readNextArc(arc);
-          arcs.add(copyArc(arc));
+          arcs.add(new FST.Arc<T>().copyFrom(arc));
         }
       
         // pick one
         arc = arcs.get(random.nextInt(arcs.size()));
-
         arcs.clear();
 
+        // accumulate output
+        output = fst.outputs.add(output, arc.output);
+
         // append label
+        if (arc.label == FST.END_LABEL) {
+          break;
+        }
+
         if (in.ints.length == in.length) {
           in.grow(1+in.length);
         }
         in.ints[in.length++] = arc.label;
-
-        output = fst.outputs.add(output, arc.output);
-
-        // maybe stop
-        if (arc.isFinal()) {
-          if (fst.hasArcs(arc.target)) {
-            // final state but it also has outgoing edges
-            if (random.nextBoolean()) {
-              output = fst.outputs.add(output, arc.nextFinalOutput);
-              break;
-            }
-          } else {
-            break;
-          }
-        }
-
-        node = arc.target;
       }
 
       return output;
     }
 
 
-    private FST<T> doTest(int prune1, int prune2) throws IOException {
+    FST<T> doTest(int prune1, int prune2) throws IOException {
       if (VERBOSE) {
         System.out.println("TEST: prune1=" + prune1 + " prune2=" + prune2);
       }
@@ -524,7 +444,7 @@ public class TestFSTs extends LuceneTest
 
       if (VERBOSE && pairs.size() <= 20 && fst != null) {
         PrintStream ps = new PrintStream("out.dot");
-        fst.toDot(ps);
+        Util.toDot(fst, ps);
         ps.close();
         System.out.println("SAVED out.dot");
       }
@@ -566,11 +486,19 @@ public class TestFSTs extends LuceneTest
 
       assertNotNull(fst);
 
-      // make sure all words are accepted
+      // visit valid paris in order -- make sure all words
+      // are accepted, and FSTEnum's next() steps through
+      // them correctly
+      if (VERBOSE) {
+        System.out.println("TEST: check valid terms/next()");
+      }
       {
         IntsRefFSTEnum<T> fstEnum = new IntsRefFSTEnum<T>(fst);
         for(InputOutput<T> pair : pairs) {
           IntsRef term = pair.input;
+          if (VERBOSE) {
+            System.out.println("TEST: check term=" + inputToString(inputMode, term) + " output=" + fst.outputs.outputToString(pair.output));
+          }
           Object output = run(fst, term, null);
 
           assertNotNull("term " + inputToString(inputMode, term) + " is not accepted", output);
@@ -578,8 +506,8 @@ public class TestFSTs extends LuceneTest
 
           // verify enum's next
           IntsRefFSTEnum.InputOutput<T> t = fstEnum.next();
-
-          assertEquals(term, t.input);
+          assertNotNull(t);
+          assertEquals("expected input=" + inputToString(inputMode, term) + " but fstEnum returned " + inputToString(inputMode, t.input), term, t.input);
           assertEquals(pair.output, t.output);
         }
         assertNull(fstEnum.next());
@@ -591,6 +519,9 @@ public class TestFSTs extends LuceneTest
       }
 
       // find random matching word and make sure it's valid
+      if (VERBOSE) {
+        System.out.println("TEST: verify random accepted terms");
+      }
       final IntsRef scratch = new IntsRef(10);
       for(int iter=0;iter<500*RANDOM_MULTIPLIER;iter++) {
         T output = randomAcceptedWord(fst, scratch);
@@ -598,10 +529,15 @@ public class TestFSTs extends LuceneTest
         assertEquals(termsMap.get(scratch), output);
       }
     
-      // test single IntsRefFSTEnum.advance:
-      //System.out.println("TEST: verify advance");
+      // test IntsRefFSTEnum.seek:
+      if (VERBOSE) {
+        System.out.println("TEST: verify seek");
+      }
+      IntsRefFSTEnum<T> fstEnum = new IntsRefFSTEnum<T>(fst);
       for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
-        final IntsRefFSTEnum<T> fstEnum = new IntsRefFSTEnum<T>(fst);
+        if (VERBOSE) {
+          System.out.println("TEST: iter=" + iter);
+        }
         if (random.nextBoolean()) {
           // seek to term that doesn't exist:
           while(true) {
@@ -611,15 +547,35 @@ public class TestFSTs extends LuceneTest
               pos = -(pos+1);
               // ok doesn't exist
               //System.out.println("  seek " + inputToString(inputMode, term));
-              final IntsRefFSTEnum.InputOutput<T> seekResult = fstEnum.advance(term);
-              if (pos < pairs.size()) {
+              final IntsRefFSTEnum.InputOutput<T> seekResult;
+              if (random.nextBoolean()) {
+                if (VERBOSE) {
+                  System.out.println("  do non-exist seekFloor term=" + inputToString(inputMode, term));
+                }
+                seekResult = fstEnum.seekFloor(term);
+                pos--;
+              } else {
+                if (VERBOSE) {
+                  System.out.println("  do non-exist seekCeil term=" + inputToString(inputMode, term));
+                }
+                seekResult = fstEnum.seekCeil(term);
+              }
+
+              if (pos != -1 && pos < pairs.size()) {
                 //System.out.println("    got " + inputToString(inputMode,seekResult.input) + " output=" + fst.outputs.outputToString(seekResult.output));
-                assertEquals(pairs.get(pos).input, seekResult.input);
+                assertNotNull("got null but expected term=" + inputToString(inputMode, pairs.get(pos).input), seekResult);
+                if (VERBOSE) {
+                  System.out.println("    got " + inputToString(inputMode, seekResult.input));
+                }
+                assertEquals("expected " + inputToString(inputMode, pairs.get(pos).input) + " but got " + inputToString(inputMode, seekResult.input), pairs.get(pos).input, seekResult.input);
                 assertEquals(pairs.get(pos).output, seekResult.output);
               } else {
-                // seeked beyond end
+                // seeked before start or beyond end
                 //System.out.println("seek=" + seekTerm);
                 assertNull("expected null but got " + (seekResult==null ? "null" : inputToString(inputMode, seekResult.input)), seekResult);
+                if (VERBOSE) {
+                  System.out.println("    got null");
+                }
               }
 
               break;
@@ -627,24 +583,36 @@ public class TestFSTs extends LuceneTest
           }
         } else {
           // seek to term that does exist:
-          InputOutput pair = pairs.get(random.nextInt(pairs.size()));
-          //System.out.println("  seek " + inputToString(inputMode, pair.input));
-          final IntsRefFSTEnum.InputOutput<T> seekResult = fstEnum.advance(pair.input);
-          assertEquals(pair.input, seekResult.input);
+          InputOutput<T> pair = pairs.get(random.nextInt(pairs.size()));
+          final IntsRefFSTEnum.InputOutput<T> seekResult;
+          if (random.nextBoolean()) {
+            if (VERBOSE) {
+              System.out.println("  do exists seekFloor " + inputToString(inputMode, pair.input));
+            }
+            seekResult = fstEnum.seekFloor(pair.input);
+          } else {
+            if (VERBOSE) {
+              System.out.println("  do exists seekCeil " + inputToString(inputMode, pair.input));
+            }
+            seekResult = fstEnum.seekCeil(pair.input);
+          }
+          assertNotNull(seekResult);
+          assertEquals("got " + inputToString(inputMode, seekResult.input) + " but expected " + inputToString(inputMode, pair.input), pair.input, seekResult.input);
           assertEquals(pair.output, seekResult.output);
         }
       }
 
       if (VERBOSE) {
-        System.out.println("TEST: mixed next/advance");
+        System.out.println("TEST: mixed next/seek");
       }
 
-      // test mixed next/advance
+      // test mixed next/seek
       for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
         if (VERBOSE) {
           System.out.println("TEST: iter " + iter);
         }
-        final IntsRefFSTEnum<T> fstEnum = new IntsRefFSTEnum<T>(fst);
+        // reset:
+        fstEnum = new IntsRefFSTEnum<T>(fst);
         int upto = -1;
         while(true) {
           boolean isDone = false;
@@ -660,13 +628,24 @@ public class TestFSTs extends LuceneTest
             for(;attempt<10;attempt++) {
               IntsRef term = toIntsRef(getRandomString(), inputMode);
               if (!termsMap.containsKey(term) && term.compareTo(pairs.get(upto).input) > 0) {
-                if (VERBOSE) {
-                  System.out.println("  do non-exist advance(" + inputToString(inputMode, term) + "]");
-                }
                 int pos = Collections.binarySearch(pairs, new InputOutput<T>(term, null));
                 assert pos < 0;
                 upto = -(pos+1);
-                isDone = fstEnum.advance(term) == null;
+
+                if (random.nextBoolean()) {
+                  upto--;
+                  assertTrue(upto != -1);
+                  if (VERBOSE) {
+                    System.out.println("  do non-exist seekFloor(" + inputToString(inputMode, term) + ")");
+                  }
+                  isDone = fstEnum.seekFloor(term) == null;
+                } else {
+                  if (VERBOSE) {
+                    System.out.println("  do non-exist seekCeil(" + inputToString(inputMode, term) + ")");
+                  }
+                  isDone = fstEnum.seekCeil(term) == null;
+                }
+
                 break;
               }
             }
@@ -681,10 +660,17 @@ public class TestFSTs extends LuceneTest
               upto = 0;
             }
 
-            if (VERBOSE) {
-              System.out.println("  do advance(" + inputToString(inputMode, pairs.get(upto).input) + "]");
+            if (random.nextBoolean()) {
+              if (VERBOSE) {
+                System.out.println("  do advanceCeil(" + inputToString(inputMode, pairs.get(upto).input) + ")");
+              }
+              isDone = fstEnum.seekCeil(pairs.get(upto).input) == null;
+            } else {
+              if (VERBOSE) {
+                System.out.println("  do advanceFloor(" + inputToString(inputMode, pairs.get(upto).input) + ")");
+              }
+              isDone = fstEnum.seekFloor(pairs.get(upto).input) == null;
             }
-            isDone = fstEnum.advance(pairs.get(upto).input) == null;
           }
           if (VERBOSE) {
             if (!isDone) {
@@ -701,6 +687,24 @@ public class TestFSTs extends LuceneTest
             assertFalse(isDone);
             assertEquals(pairs.get(upto).input, fstEnum.current().input);
             assertEquals(pairs.get(upto).output, fstEnum.current().output);
+
+            /*
+            if (upto < pairs.size()-1) {
+              int tryCount = 0;
+              while(tryCount < 10) {
+                final IntsRef t = toIntsRef(getRandomString(), inputMode);
+                if (pairs.get(upto).input.compareTo(t) < 0) {
+                  final boolean expected = t.compareTo(pairs.get(upto+1).input) < 0;
+                  if (VERBOSE) {
+                    System.out.println("TEST: call beforeNext(" + inputToString(inputMode, t) + "); current=" + inputToString(inputMode, pairs.get(upto).input) + " next=" + inputToString(inputMode, pairs.get(upto+1).input) + " expected=" + expected);
+                  }
+                  assertEquals(expected, fstEnum.beforeNext(t));
+                  break;
+                }
+                tryCount++;
+              }
+            }
+            */
           }
         }
       }
@@ -757,7 +761,9 @@ public class TestFSTs extends LuceneTest
         }
       }
 
-      //System.out.println("TEST: now prune");
+      if (VERBOSE) {
+        System.out.println("TEST: now prune");
+      }
 
       // prune 'em
       final Iterator<Map.Entry<IntsRef,CountMinOutput<T>>> it = prefixes.entrySet().iterator();
@@ -765,7 +771,9 @@ public class TestFSTs extends LuceneTest
         Map.Entry<IntsRef,CountMinOutput<T>> ent = it.next();
         final IntsRef prefix = ent.getKey();
         final CountMinOutput<T> cmo = ent.getValue();
-        //System.out.println("  term=" + inputToString(inputMode, prefix) + " count=" + cmo.count + " isLeaf=" + cmo.isLeaf);
+        if (VERBOSE) {
+          System.out.println("  term=" + inputToString(inputMode, prefix) + " count=" + cmo.count + " isLeaf=" + cmo.isLeaf + " output=" + outputs.outputToString(cmo.output) + " isFinal=" + cmo.isFinal);
+        }
         final boolean keep;
         if (prune1 > 0) {
           keep = cmo.count >= prune1;
@@ -824,14 +832,20 @@ public class TestFSTs extends LuceneTest
       assertNotNull(fst);
 
       // make sure FST only enums valid prefixes
+      if (VERBOSE) {
+        System.out.println("TEST: check pruned enum");
+      }
       IntsRefFSTEnum<T> fstEnum = new IntsRefFSTEnum<T>(fst);
-      IntsRefFSTEnum.InputOutput current;
+      IntsRefFSTEnum.InputOutput<T> current;
       while((current = fstEnum.next()) != null) {
-        //System.out.println("  fst enum term=" + inputToString(inputMode, current.input) + " output=" + outputs.outputToString(current.output));
+        if (VERBOSE) {
+          System.out.println("  fstEnum.next term=" + inputToString(inputMode, current.input) + " output=" + outputs.outputToString(current.output));
+        }
         final CountMinOutput cmo = prefixes.get(current.input);
         assertNotNull(cmo);
         assertTrue(cmo.isLeaf || cmo.isFinal);
-        if (cmo.isFinal && !cmo.isLeaf) {
+        //if (cmo.isFinal && !cmo.isLeaf) {
+        if (cmo.isFinal) {
           assertEquals(cmo.finalOutput, current.output);
         } else {
           assertEquals(cmo.output, current.output);
@@ -839,19 +853,24 @@ public class TestFSTs extends LuceneTest
       }
 
       // make sure all non-pruned prefixes are present in the FST
-      final int[] stopNode = new int[2];
+      if (VERBOSE) {
+        System.out.println("TEST: verify all prefixes");
+      }
+      final int[] stopNode = new int[1];
       for(Map.Entry<IntsRef,CountMinOutput<T>> ent : prefixes.entrySet()) {
         if (ent.getKey().length > 0) {
           final CountMinOutput<T> cmo = ent.getValue();
           final T output = run(fst, ent.getKey(), stopNode);
-          //System.out.println("  term=" + inputToString(inputMode, ent.getKey()) + " output=" + outputs.outputToString(cmo.output));
+          if (VERBOSE) {
+            System.out.println("TEST: verify term=" + inputToString(inputMode, ent.getKey()) + " output=" + outputs.outputToString(cmo.output));
+          }
           // if (cmo.isFinal && !cmo.isLeaf) {
           if (cmo.isFinal) {
             assertEquals(cmo.finalOutput, output);
           } else {
             assertEquals(cmo.output, output);
           }
-          assertEquals(ent.getKey().length, stopNode[1]);
+          assertEquals(ent.getKey().length, stopNode[0]);
         }
       }
     }
@@ -859,7 +878,7 @@ public class TestFSTs extends LuceneTest
 
   public void testRandomWords() throws IOException {
     testRandomWords(1000, 5 * RANDOM_MULTIPLIER);
-    //testRandomWords(10, 100);
+    //testRandomWords(20, 100);
   }
 
   private String inputModeToString(int mode) {
@@ -888,7 +907,7 @@ public class TestFSTs extends LuceneTest
     }
   }
 
-  private String getRandomString() {
+  static String getRandomString() {
     final String term;
     if (random.nextBoolean()) {
       term = _TestUtil.randomRealisticUnicodeString(random);
@@ -909,10 +928,10 @@ public class TestFSTs extends LuceneTest
   private static String inputToString(int inputMode, IntsRef term) {
     if (inputMode == 0) {
       // utf8
-      return toBytesRef(term).utf8ToString();
+      return toBytesRef(term).utf8ToString() + " " + term;
     } else {
       // utf32
-      return UnicodeUtil.newString(term.ints, term.offset, term.length);
+      return UnicodeUtil.newString(term.ints, term.offset, term.length) + " " + term;
     }
   }
 
@@ -931,6 +950,7 @@ public class TestFSTs extends LuceneTest
     final File tempDir = _TestUtil.getTempDir("fstlines");
     final MockDirectoryWrapper dir = new MockDirectoryWrapper(random, FSDirectory.open(tempDir));
     final IndexWriter writer = new IndexWriter(dir, conf);
+    writer.setInfoStream(VERBOSE ? System.out : null);
     final long stopTime = System.currentTimeMillis() + RUN_TIME_SEC * 1000;
     Document doc;
     int docCount = 0;
@@ -986,18 +1006,17 @@ public class TestFSTs extends LuceneTest
         // same:
         final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<Long>(fst);
         for(int iter=0;iter<1000*RANDOM_MULTIPLIER;iter++) {
-          fstEnum.reset();
           final BytesRef randomTerm = new BytesRef(getRandomString());
         
-          final TermsEnum.SeekStatus seekResult = termsEnum.seek(randomTerm);
-          final BytesRefFSTEnum.InputOutput fstSeekResult = fstEnum.advance(randomTerm);
-
           if (VERBOSE) {
-            System.out.println("TEST: seek " + randomTerm.utf8ToString());
+            System.out.println("TEST: seek " + randomTerm.utf8ToString() + " " + randomTerm);
           }
 
+          final TermsEnum.SeekStatus seekResult = termsEnum.seek(randomTerm);
+          final BytesRefFSTEnum.InputOutput fstSeekResult = fstEnum.seekCeil(randomTerm);
+
           if (seekResult == TermsEnum.SeekStatus.END) {
-            assertNull(fstSeekResult);
+            assertNull("got " + (fstSeekResult == null ? "null" : fstSeekResult.input.utf8ToString()) + " but expected null", fstSeekResult);
           } else {
             assertSame(termsEnum, fstEnum, storeOrd);
             for(int nextIter=0;nextIter<10;nextIter++) {
@@ -1011,6 +1030,9 @@ public class TestFSTs extends LuceneTest
                 assertNotNull(fstEnum.next());
                 assertSame(termsEnum, fstEnum, storeOrd);
               } else {
+                if (VERBOSE) {
+                  System.out.println("  end!");
+                }
                 BytesRefFSTEnum.InputOutput<Long> nextResult = fstEnum.next();
                 if (nextResult != null) {
                   System.out.println("expected null but got: input=" + nextResult.input.utf8ToString() + " output=" + outputs.outputToString(nextResult.output));
@@ -1032,7 +1054,8 @@ public class TestFSTs extends LuceneTest
     if (termsEnum.term() == null) {
       assertNull(fstEnum.current());
     } else {
-      assertEquals(termsEnum.term(), fstEnum.current().input);
+      assertNotNull(fstEnum.current());
+      assertEquals(termsEnum.term().utf8ToString() + " != " + fstEnum.current().input.utf8ToString(), termsEnum.term(), fstEnum.current().input);
       if (storeOrd) {
         // fst stored the ord
         assertEquals(termsEnum.ord(), ((Long) fstEnum.current().output).longValue());
@@ -1095,7 +1118,7 @@ public class TestFSTs extends LuceneTest
         System.out.println(ord + " terms; " + fst.getNodeCount() + " nodes; " + fst.getArcCount() + " arcs; " + fst.getArcWithOutputCount() + " arcs w/ output; tot size " + fst.sizeInBytes());
         if (fst.getNodeCount() < 100) {
           PrintStream ps = new PrintStream("out.dot");
-          fst.toDot(ps);
+          Util.toDot(fst, ps);
           ps.close();
           System.out.println("Wrote FST to out.dot");
         }
@@ -1121,7 +1144,7 @@ public class TestFSTs extends LuceneTest
           }
           toIntsRef(w, inputMode, intsRef);
           T expected = getOutput(intsRef, ord);
-          T actual = fst.get(intsRef);
+          T actual = Util.get(fst, intsRef);
           if (actual == null) {
             throw new RuntimeException("unexpected null output on input=" + w);
           }
@@ -1233,4 +1256,57 @@ public class TestFSTs extends LuceneTest
       }.run(limit);
     }
   }
+
+  public void testSingleString() throws Exception {
+    final Outputs<Object> outputs = NoOutputs.getSingleton();
+    final Builder<Object> b = new Builder<Object>(FST.INPUT_TYPE.BYTE1, 0, 0, true, outputs);
+    b.add(new BytesRef("foobar"), outputs.getNoOutput());
+    final BytesRefFSTEnum<Object> fstEnum = new BytesRefFSTEnum<Object>(b.finish());
+    assertNull(fstEnum.seekFloor(new BytesRef("foo")));
+    assertNull(fstEnum.seekCeil(new BytesRef("foobaz")));
+  }
+
+  public void testSimple() throws Exception {
+
+    // Get outputs -- passing true means FST will share
+    // (delta code) the outputs.  This should result in
+    // smaller FST if the outputs grow monotonically.  But
+    // if numbers are "random", false should give smaller
+    // final size:
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+
+    // Build an FST mapping BytesRef -> Long
+    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, outputs);
+
+    final BytesRef a = new BytesRef("a");
+    final BytesRef b = new BytesRef("b");
+    final BytesRef c = new BytesRef("c");
+
+    builder.add(a, outputs.get(17));
+    builder.add(b, outputs.get(42));
+    builder.add(c, outputs.get(13824324872317238L));
+
+    final FST<Long> fst = builder.finish();
+
+    assertEquals(13824324872317238L, (long) Util.get(fst, c));
+    assertEquals(42, (long) Util.get(fst, b));
+    assertEquals(17, (long) Util.get(fst, a));
+
+    BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<Long>(fst);
+    BytesRefFSTEnum.InputOutput<Long> seekResult;
+    seekResult = fstEnum.seekFloor(a);
+    assertNotNull(seekResult);
+    assertEquals(17, (long) seekResult.output);
+
+    // goes to a
+    seekResult = fstEnum.seekFloor(new BytesRef("aa"));
+    assertNotNull(seekResult);
+    assertEquals(17, (long) seekResult.output);
+
+    // goes to b
+    seekResult = fstEnum.seekCeil(new BytesRef("aa"));
+    assertNotNull(seekResult);
+    assertEquals(b, seekResult.input);
+    assertEquals(42, (long) seekResult.output);
+  }
 }

Modified: lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/ar/ArabicAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/ar/ArabicAnalyzer.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/ar/ArabicAnalyzer.java (original)
+++ lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/ar/ArabicAnalyzer.java Wed Jan  5 20:25:17 2011
@@ -17,10 +17,8 @@ package org.apache.lucene.analysis.ar;
  * limitations under the License.
  */
 
-import java.io.File;
 import java.io.IOException;
 import java.io.Reader;
-import java.util.Hashtable;
 import java.util.Set;
 
 import org.apache.lucene.analysis.Analyzer;
@@ -30,7 +28,6 @@ import org.apache.lucene.analysis.miscel
 import org.apache.lucene.analysis.standard.StandardTokenizer;
 import org.apache.lucene.analysis.util.CharArraySet;
 import org.apache.lucene.analysis.util.StopwordAnalyzerBase;
-import org.apache.lucene.analysis.util.WordlistLoader;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.util.Version;

Modified: lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsFilter.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsFilter.java (original)
+++ lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsFilter.java Wed Jan  5 20:25:17 2011
@@ -10,7 +10,6 @@
 package org.apache.lucene.analysis.commongrams;
 
 import java.io.IOException;
-import java.util.Arrays;
 import java.util.Set;
 
 import org.apache.lucene.analysis.TokenFilter;

Modified: lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/hyphenation/PatternParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/hyphenation/PatternParser.java?rev=1055622&r1=1055621&r2=1055622&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/hyphenation/PatternParser.java (original)
+++ lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/hyphenation/PatternParser.java Wed Jan  5 20:25:17 2011
@@ -29,7 +29,6 @@ import org.xml.sax.Attributes;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.io.StringReader;
 import java.net.MalformedURLException;
 import java.util.ArrayList;