You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2014/04/05 11:03:00 UTC

svn commit: r1585024 [3/4] - in /lucene/dev/branches/solr5914: ./ dev-tools/ dev-tools/idea/.idea/libraries/ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/index/ lucene/core/src/java/org/apache/lucene/search/ lucene/core/src/java/org/apac...

Modified: lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java (original)
+++ lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java Sat Apr  5 09:02:57 2014
@@ -22,6 +22,7 @@ import java.util.Collection;
 
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.search.LeafCollector;
 import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.grouping.AbstractSecondPassGroupingCollector;
@@ -53,8 +54,8 @@ public class TermSecondPassGroupingColle
   }
 
   @Override
-  public void setNextReader(AtomicReaderContext readerContext) throws IOException {
-    super.setNextReader(readerContext);
+  protected void doSetNextReader(AtomicReaderContext readerContext) throws IOException {
+    super.doSetNextReader(readerContext);
     index = FieldCache.DEFAULT.getTermsIndex(readerContext.reader(), groupField);
 
     // Rebuild ordSet

Modified: lucene/dev/branches/solr5914/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterPhraseTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterPhraseTest.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterPhraseTest.java (original)
+++ lucene/dev/branches/solr5914/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterPhraseTest.java Sat Apr  5 09:02:57 2014
@@ -35,12 +35,13 @@ import org.apache.lucene.index.Directory
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.search.LeafCollector;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SimpleCollector;
 import org.apache.lucene.search.TopDocs;
-
 import org.apache.lucene.search.spans.SpanNearQuery;
 import org.apache.lucene.search.spans.SpanQuery;
 import org.apache.lucene.search.spans.SpanTermQuery;
@@ -116,7 +117,7 @@ public class HighlighterPhraseTest exten
           new SpanTermQuery(new Term(FIELD, "fox")),
           new SpanTermQuery(new Term(FIELD, "jumped")) }, 0, true);
       final FixedBitSet bitset = new FixedBitSet(indexReader.maxDoc());
-      indexSearcher.search(phraseQuery, new Collector() {
+      indexSearcher.search(phraseQuery, new SimpleCollector() {
         private int baseDoc;
 
         @Override
@@ -130,7 +131,7 @@ public class HighlighterPhraseTest exten
         }
 
         @Override
-        public void setNextReader(AtomicReaderContext context) {
+        protected void doSetNextReader(AtomicReaderContext context) throws IOException {
           this.baseDoc = context.docBase;
         }
 

Modified: lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/FakeScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/FakeScorer.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/FakeScorer.java (original)
+++ lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/FakeScorer.java Sat Apr  5 09:02:57 2014
@@ -19,11 +19,11 @@ package org.apache.lucene.search.join;
 
 import java.util.Collection;
 
-import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.LeafCollector;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
 
-/** Passed to {@link Collector#setScorer} during join collection. */
+/** Passed to {@link LeafCollector#setScorer} during join collection. */
 final class FakeScorer extends Scorer {
   float score;
   int doc = -1;

Modified: lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/TermsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/TermsCollector.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/TermsCollector.java (original)
+++ lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/TermsCollector.java Sat Apr  5 09:02:57 2014
@@ -22,9 +22,11 @@ import java.io.IOException;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.search.LeafCollector;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.SimpleCollector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
 
@@ -33,7 +35,7 @@ import org.apache.lucene.util.BytesRefHa
  *
  * @lucene.experimental
  */
-abstract class TermsCollector extends Collector {
+abstract class TermsCollector extends SimpleCollector {
 
   final String field;
   final BytesRefHash collectorTerms = new BytesRefHash();
@@ -47,10 +49,6 @@ abstract class TermsCollector extends Co
   }
 
   @Override
-  public void setScorer(Scorer scorer) throws IOException {
-  }
-
-  @Override
   public boolean acceptsDocsOutOfOrder() {
     return true;
   }
@@ -86,7 +84,7 @@ abstract class TermsCollector extends Co
     }
 
     @Override
-    public void setNextReader(AtomicReaderContext context) throws IOException {
+    protected void doSetNextReader(AtomicReaderContext context) throws IOException {
       docTermOrds = FieldCache.DEFAULT.getDocTermOrds(context.reader(), field);
     }
   }
@@ -108,7 +106,7 @@ abstract class TermsCollector extends Co
     }
 
     @Override
-    public void setNextReader(AtomicReaderContext context) throws IOException {
+    protected void doSetNextReader(AtomicReaderContext context) throws IOException {
       fromDocTerms = FieldCache.DEFAULT.getTerms(context.reader(), field, false);
     }
   }

Modified: lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java (original)
+++ lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java Sat Apr  5 09:02:57 2014
@@ -27,6 +27,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.LeafCollector;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.ComplexExplanation;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -227,7 +228,7 @@ class TermsIncludingScoreQuery extends Q
     }
 
     @Override
-    public boolean score(Collector collector, int max) throws IOException {
+    public boolean score(LeafCollector collector, int max) throws IOException {
       FakeScorer fakeScorer = new FakeScorer();
       collector.setScorer(fakeScorer);
       if (doc == -1) {

Modified: lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/TermsWithScoreCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/TermsWithScoreCollector.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/TermsWithScoreCollector.java (original)
+++ lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/TermsWithScoreCollector.java Sat Apr  5 09:02:57 2014
@@ -22,14 +22,16 @@ import java.io.IOException;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.search.LeafCollector;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.SimpleCollector;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
 
-abstract class TermsWithScoreCollector extends Collector {
+abstract class TermsWithScoreCollector extends SimpleCollector {
 
   private final static int INITIAL_ARRAY_SIZE = 256;
 
@@ -128,7 +130,7 @@ abstract class TermsWithScoreCollector e
     }
 
     @Override
-    public void setNextReader(AtomicReaderContext context) throws IOException {
+    protected void doSetNextReader(AtomicReaderContext context) throws IOException {
       fromDocTerms = FieldCache.DEFAULT.getTerms(context.reader(), field, false);
     }
 
@@ -214,7 +216,7 @@ abstract class TermsWithScoreCollector e
     }
 
     @Override
-    public void setNextReader(AtomicReaderContext context) throws IOException {
+    protected void doSetNextReader(AtomicReaderContext context) throws IOException {
       fromDocTermOrds = FieldCache.DEFAULT.getDocTermOrds(context.reader(), field);
     }
 

Modified: lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java (original)
+++ lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java Sat Apr  5 09:02:57 2014
@@ -74,7 +74,7 @@ import java.util.*;
  *
  * @lucene.experimental
  */
-public class ToParentBlockJoinCollector extends Collector {
+public class ToParentBlockJoinCollector extends SimpleCollector {
 
   private final Sort sort;
 
@@ -269,7 +269,7 @@ public class ToParentBlockJoinCollector 
   }
 
   @Override
-  public void setNextReader(AtomicReaderContext context) throws IOException {
+  protected void doSetNextReader(AtomicReaderContext context) throws IOException {
     currentReaderContext = context;
     docBase = context.docBase;
     for (int compIDX = 0; compIDX < comparators.length; compIDX++) {
@@ -421,7 +421,7 @@ public class ToParentBlockJoinCollector 
       }
 
       collector.setScorer(fakeScorer);
-      collector.setNextReader(og.readerContext);
+      collector.getLeafCollector(og.readerContext);
       for(int docIDX=0;docIDX<numChildDocs;docIDX++) {
         //System.out.println("docIDX=" + docIDX + " vs " + og.docs[slot].length);
         final int doc = og.docs[slot][docIDX];

Modified: lucene/dev/branches/solr5914/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java (original)
+++ lucene/dev/branches/solr5914/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java Sat Apr  5 09:02:57 2014
@@ -47,6 +47,7 @@ import org.apache.lucene.index.SortedSet
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.LeafCollector;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.Collector;
@@ -58,6 +59,7 @@ import org.apache.lucene.search.MatchAll
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.SimpleCollector;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.TopScoreDocCollector;
@@ -222,12 +224,9 @@ public class TestJoinUtil extends Lucene
     bq.add(joinQuery, BooleanClause.Occur.SHOULD);
     bq.add(new TermQuery(new Term("id", "3")), BooleanClause.Occur.SHOULD);
 
-    indexSearcher.search(bq, new Collector() {
+    indexSearcher.search(bq, new SimpleCollector() {
         boolean sawFive;
         @Override
-        public void setNextReader(AtomicReaderContext context) {
-        }
-        @Override
         public void collect(int docID) {
           // Hairy / evil (depends on how BooleanScorer
           // stores temporarily collected docIDs by
@@ -239,9 +238,6 @@ public class TestJoinUtil extends Lucene
           }
         }
         @Override
-        public void setScorer(Scorer scorer) {
-        }
-        @Override
         public boolean acceptsDocsOutOfOrder() {
           return true;
         }
@@ -407,7 +403,7 @@ public class TestJoinUtil extends Lucene
         // Need to know all documents that have matches. TopDocs doesn't give me that and then I'd be also testing TopDocsCollector...
         final FixedBitSet actualResult = new FixedBitSet(indexSearcher.getIndexReader().maxDoc());
         final TopScoreDocCollector topScoreDocCollector = TopScoreDocCollector.create(10, false);
-        indexSearcher.search(joinQuery, new Collector() {
+        indexSearcher.search(joinQuery, new SimpleCollector() {
 
           int docBase;
 
@@ -418,9 +414,9 @@ public class TestJoinUtil extends Lucene
           }
 
           @Override
-          public void setNextReader(AtomicReaderContext context) {
+          protected void doSetNextReader(AtomicReaderContext context) throws IOException {
             docBase = context.docBase;
-            topScoreDocCollector.setNextReader(context);
+            topScoreDocCollector.getLeafCollector(context);
           }
 
           @Override
@@ -572,7 +568,7 @@ public class TestJoinUtil extends Lucene
       }
       final Map<BytesRef, JoinScore> joinValueToJoinScores = new HashMap<>();
       if (multipleValuesPerDocument) {
-        fromSearcher.search(new TermQuery(new Term("value", uniqueRandomValue)), new Collector() {
+        fromSearcher.search(new TermQuery(new Term("value", uniqueRandomValue)), new SimpleCollector() {
 
           private Scorer scorer;
           private SortedSetDocValues docTermOrds;
@@ -593,7 +589,7 @@ public class TestJoinUtil extends Lucene
           }
 
           @Override
-          public void setNextReader(AtomicReaderContext context) throws IOException {
+          protected void doSetNextReader(AtomicReaderContext context) throws IOException {
             docTermOrds = FieldCache.DEFAULT.getDocTermOrds(context.reader(), fromField);
           }
 
@@ -608,7 +604,7 @@ public class TestJoinUtil extends Lucene
           }
         });
       } else {
-        fromSearcher.search(new TermQuery(new Term("value", uniqueRandomValue)), new Collector() {
+        fromSearcher.search(new TermQuery(new Term("value", uniqueRandomValue)), new SimpleCollector() {
 
           private Scorer scorer;
           private BinaryDocValues terms;
@@ -631,7 +627,7 @@ public class TestJoinUtil extends Lucene
           }
 
           @Override
-          public void setNextReader(AtomicReaderContext context) throws IOException {
+          protected void doSetNextReader(AtomicReaderContext context) throws IOException {
             terms = FieldCache.DEFAULT.getTerms(context.reader(), fromField, true);
             docsWithField = FieldCache.DEFAULT.getDocsWithField(context.reader(), fromField);
           }
@@ -675,7 +671,7 @@ public class TestJoinUtil extends Lucene
             }
           }
         } else {
-          toSearcher.search(new MatchAllDocsQuery(), new Collector() {
+          toSearcher.search(new MatchAllDocsQuery(), new SimpleCollector() {
 
             private SortedSetDocValues docTermOrds;
             private final BytesRef scratch = new BytesRef();
@@ -701,7 +697,7 @@ public class TestJoinUtil extends Lucene
             }
 
             @Override
-            public void setNextReader(AtomicReaderContext context) throws IOException {
+            protected void doSetNextReader(AtomicReaderContext context) throws IOException {
               docBase = context.docBase;
               docTermOrds = FieldCache.DEFAULT.getDocTermOrds(context.reader(), toField);
             }
@@ -713,7 +709,7 @@ public class TestJoinUtil extends Lucene
           });
         }
       } else {
-        toSearcher.search(new MatchAllDocsQuery(), new Collector() {
+        toSearcher.search(new MatchAllDocsQuery(), new SimpleCollector() {
 
           private BinaryDocValues terms;
           private int docBase;
@@ -730,7 +726,7 @@ public class TestJoinUtil extends Lucene
           }
 
           @Override
-          public void setNextReader(AtomicReaderContext context) throws IOException {
+          protected void doSetNextReader(AtomicReaderContext context) throws IOException {
             terms = FieldCache.DEFAULT.getTerms(context.reader(), toField, false);
             docBase = context.docBase;
           }

Modified: lucene/dev/branches/solr5914/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (original)
+++ lucene/dev/branches/solr5914/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java Sat Apr  5 09:02:57 2014
@@ -54,6 +54,7 @@ import org.apache.lucene.search.Collecto
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.SimpleCollector;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.store.RAMDirectory; // for javadocs
 import org.apache.lucene.util.ArrayUtil;
@@ -532,7 +533,7 @@ public class MemoryIndex {
     IndexSearcher searcher = createSearcher();
     try {
       final float[] scores = new float[1]; // inits to 0.0f (no match)
-      searcher.search(query, new Collector() {
+      searcher.search(query, new SimpleCollector() {
         private Scorer scorer;
 
         @Override
@@ -550,8 +551,6 @@ public class MemoryIndex {
           return true;
         }
 
-        @Override
-        public void setNextReader(AtomicReaderContext context) { }
       });
       float score = scores[0];
       return score;

Modified: lucene/dev/branches/solr5914/lucene/misc/src/java/org/apache/lucene/index/sorter/EarlyTerminatingSortingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/misc/src/java/org/apache/lucene/index/sorter/EarlyTerminatingSortingCollector.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/misc/src/java/org/apache/lucene/index/sorter/EarlyTerminatingSortingCollector.java (original)
+++ lucene/dev/branches/solr5914/lucene/misc/src/java/org/apache/lucene/index/sorter/EarlyTerminatingSortingCollector.java Sat Apr  5 09:02:57 2014
@@ -21,9 +21,11 @@ import java.io.IOException;
 
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.search.LeafCollector;
 import org.apache.lucene.search.CollectionTerminatedException;
 import org.apache.lucene.search.Collector;
-import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.FilterLeafCollector;
+import org.apache.lucene.search.FilterCollector;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.TopDocsCollector;
 import org.apache.lucene.search.TotalHitCountCollector;
@@ -32,11 +34,11 @@ import org.apache.lucene.search.TotalHit
  * A {@link Collector} that early terminates collection of documents on a
  * per-segment basis, if the segment was sorted according to the given
  * {@link Sort}.
- * 
+ *
  * <p>
  * <b>NOTE:</b> the {@code Collector} detects sorted segments according to
  * {@link SortingMergePolicy}, so it's best used in conjunction with it. Also,
- * it collects up to a specified {@code numDocsToCollect} from each segment, 
+ * it collects up to a specified {@code numDocsToCollect} from each segment,
  * and therefore is mostly suitable for use in conjunction with collectors such as
  * {@link TopDocsCollector}, and not e.g. {@link TotalHitCountCollector}.
  * <p>
@@ -58,26 +60,21 @@ import org.apache.lucene.search.TotalHit
  * the old and the new {@code Sort}s have the same identifier, this
  * {@code Collector} will incorrectly detect sorted segments.</li>
  * </ul>
- * 
+ *
  * @lucene.experimental
  */
-public class EarlyTerminatingSortingCollector extends Collector {
-  /** The wrapped Collector */
-  protected final Collector in;
+public class EarlyTerminatingSortingCollector extends FilterCollector {
+
   /** Sort used to sort the search results */
   protected final Sort sort;
   /** Number of documents to collect in each segment */
   protected final int numDocsToCollect;
-  /** Number of documents to collect in the current segment being processed */
-  protected int segmentTotalCollect;
-  /** True if the current segment being processed is sorted by {@link #sort} */
-  protected boolean segmentSorted;
 
   private int numCollected;
 
   /**
    * Create a new {@link EarlyTerminatingSortingCollector} instance.
-   * 
+   *
    * @param in
    *          the collector to wrap
    * @param sort
@@ -88,38 +85,37 @@ public class EarlyTerminatingSortingColl
    *          hits.
    */
   public EarlyTerminatingSortingCollector(Collector in, Sort sort, int numDocsToCollect) {
+    super(in);
     if (numDocsToCollect <= 0) {
-      throw new IllegalStateException("numDocsToCollect must always be > 0, got " + segmentTotalCollect);
+      throw new IllegalStateException("numDocsToCollect must always be > 0, got " + numDocsToCollect);
     }
-    this.in = in;
     this.sort = sort;
     this.numDocsToCollect = numDocsToCollect;
   }
 
   @Override
-  public void setScorer(Scorer scorer) throws IOException {
-    in.setScorer(scorer);
-  }
-
-  @Override
-  public void collect(int doc) throws IOException {
-    in.collect(doc);
-    if (++numCollected >= segmentTotalCollect) {
-      throw new CollectionTerminatedException();
+  public LeafCollector getLeafCollector(AtomicReaderContext context) throws IOException {
+    if (SortingMergePolicy.isSorted(context.reader(), sort)) {
+      // segment is sorted, can early-terminate
+      return new FilterLeafCollector(super.getLeafCollector(context)) {
+
+        @Override
+        public void collect(int doc) throws IOException {
+          super.collect(doc);
+          if (++numCollected >= numDocsToCollect) {
+            throw new CollectionTerminatedException();
+          }
+        }
+
+        @Override
+        public boolean acceptsDocsOutOfOrder() {
+          return false;
+        }
+
+      };
+    } else {
+      return super.getLeafCollector(context);
     }
   }
 
-  @Override
-  public void setNextReader(AtomicReaderContext context) throws IOException {
-    in.setNextReader(context);
-    segmentSorted = SortingMergePolicy.isSorted(context.reader(), sort);
-    segmentTotalCollect = segmentSorted ? numDocsToCollect : Integer.MAX_VALUE;
-    numCollected = 0;
-  }
-
-  @Override
-  public boolean acceptsDocsOutOfOrder() {
-    return !segmentSorted && in.acceptsDocsOutOfOrder();
-  }
-
 }

Modified: lucene/dev/branches/solr5914/lucene/misc/src/test/org/apache/lucene/index/sorter/TestEarlyTermination.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/misc/src/test/org/apache/lucene/index/sorter/TestEarlyTermination.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/misc/src/test/org/apache/lucene/index/sorter/TestEarlyTermination.java (original)
+++ lucene/dev/branches/solr5914/lucene/misc/src/test/org/apache/lucene/index/sorter/TestEarlyTermination.java Sat Apr  5 09:02:57 2014
@@ -34,6 +34,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.search.LeafCollector;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Sort;
@@ -147,9 +148,10 @@ public class TestEarlyTermination extend
       Sort different = new Sort(new SortField("ndv2", SortField.Type.LONG));
       searcher.search(query, new EarlyTerminatingSortingCollector(collector2, different, numHits) {
         @Override
-        public void setNextReader(AtomicReaderContext context) throws IOException {
-          super.setNextReader(context);
-          assertFalse("segment should not be recognized as sorted as different sorter was used", segmentSorted);
+        public LeafCollector getLeafCollector(AtomicReaderContext context) throws IOException {
+          final LeafCollector ret = super.getLeafCollector(context);
+          assertTrue("segment should not be recognized as sorted as different sorter was used", ret.getClass() == in.getLeafCollector(context).getClass());
+          return ret;
         }
       });
     }

Modified: lucene/dev/branches/solr5914/lucene/queryparser/src/test/org/apache/lucene/queryparser/surround/query/BooleanQueryTst.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/queryparser/src/test/org/apache/lucene/queryparser/surround/query/BooleanQueryTst.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/queryparser/src/test/org/apache/lucene/queryparser/surround/query/BooleanQueryTst.java (original)
+++ lucene/dev/branches/solr5914/lucene/queryparser/src/test/org/apache/lucene/queryparser/surround/query/BooleanQueryTst.java Sat Apr  5 09:02:57 2014
@@ -22,13 +22,13 @@ import java.io.IOException;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.LeafCollector;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Query;
-
+import org.apache.lucene.search.SimpleCollector;
 import org.apache.lucene.queryparser.surround.parser.QueryParser;
-
 import org.junit.Assert;
 
 public class BooleanQueryTst {
@@ -57,7 +57,7 @@ public class BooleanQueryTst {
   
   public void setVerbose(boolean verbose) {this.verbose = verbose;}
 
-  class TestCollector extends Collector { // FIXME: use check hits from Lucene tests
+  class TestCollector extends SimpleCollector { // FIXME: use check hits from Lucene tests
     int totalMatched;
     boolean[] encountered;
     private Scorer scorer = null;
@@ -79,7 +79,7 @@ public class BooleanQueryTst {
     }
 
     @Override
-    public void setNextReader(AtomicReaderContext context) throws IOException {
+    protected void doSetNextReader(AtomicReaderContext context) throws IOException {
       docBase = context.docBase;
     }
     

Modified: lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java (original)
+++ lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java Sat Apr  5 09:02:57 2014
@@ -25,7 +25,6 @@ import org.apache.lucene.spatial.prefix.
 import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.StringHelper;
 
 import java.io.IOException;
 import java.util.Iterator;
@@ -117,7 +116,7 @@ public abstract class AbstractVisitingPr
     protected final boolean hasIndexedLeaves;//if false then we can skip looking for them
 
     private VNode curVNode;//current pointer, derived from query shape
-    private BytesRef curVNodeTerm = new BytesRef();//curVNode.cell's term.
+    private BytesRef curVNodeTerm = new BytesRef();//curVNode.cell's term, without leaf
     private Cell scanCell;
 
     private BytesRef thisTerm;//the result of termsEnum.term()
@@ -171,8 +170,7 @@ public abstract class AbstractVisitingPr
         }
 
         //Seek to curVNode's cell (or skip if termsEnum has moved beyond)
-        curVNodeTerm.bytes = curVNode.cell.getTokenBytes();
-        curVNodeTerm.length = curVNodeTerm.bytes.length;
+        curVNode.cell.getTokenBytesNoLeaf(curVNodeTerm);
         int compare = thisTerm.compareTo(curVNodeTerm);
         if (compare > 0) {
           // leap frog (termsEnum is beyond where we would otherwise seek)
@@ -215,7 +213,7 @@ public abstract class AbstractVisitingPr
       if (hasIndexedLeaves && cell.getLevel() != 0) {
         //If the next indexed term just adds a leaf marker ('+') to cell,
         // then add all of those docs
-        assert StringHelper.startsWith(thisTerm, curVNodeTerm);//TODO refactor to use method on curVNode.cell
+        assert curVNode.cell.isWithin(curVNodeTerm, thisTerm);
         scanCell = grid.getCell(thisTerm.bytes, thisTerm.offset, thisTerm.length, scanCell);
         if (scanCell.getLevel() == cell.getLevel() && scanCell.isLeaf()) {
           visitLeaf(scanCell);
@@ -265,7 +263,7 @@ public abstract class AbstractVisitingPr
      */
     protected void scan(int scanDetailLevel) throws IOException {
       for (;
-           thisTerm != null && StringHelper.startsWith(thisTerm, curVNodeTerm);//TODO refactor to use method on curVNode.cell
+           thisTerm != null && curVNode.cell.isWithin(curVNodeTerm, thisTerm);
            thisTerm = termsEnum.next()) {
         scanCell = grid.getCell(thisTerm.bytes, thisTerm.offset, thisTerm.length, scanCell);
 

Modified: lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java (original)
+++ lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java Sat Apr  5 09:02:57 2014
@@ -83,7 +83,7 @@ public class ContainsPrefixTreeFilter ex
       super(context, acceptDocs);
     }
 
-    BytesRef termBytes = new BytesRef();
+    BytesRef termBytes = new BytesRef();//no leaf
     Cell nextCell;//see getLeafDocs
 
     /** This is the primary algorithm; recursive.  Returns null if finds none. */
@@ -130,16 +130,15 @@ public class ContainsPrefixTreeFilter ex
     }
 
     private boolean seekExact(Cell cell) throws IOException {
-      assert new BytesRef(cell.getTokenBytes()).compareTo(termBytes) > 0;
-      termBytes.bytes = cell.getTokenBytes();
-      termBytes.length = termBytes.bytes.length;
+      assert cell.getTokenBytesNoLeaf(null).compareTo(termBytes) > 0;
+      cell.getTokenBytesNoLeaf(termBytes);
       if (termsEnum == null)
         return false;
       return termsEnum.seekExact(termBytes);
     }
 
     private SmallDocSet getDocs(Cell cell, Bits acceptContains) throws IOException {
-      assert new BytesRef(cell.getTokenBytes()).equals(termBytes);
+      assert cell.getTokenBytesNoLeaf(null).equals(termBytes);
 
       return collectDocs(acceptContains);
     }
@@ -147,7 +146,7 @@ public class ContainsPrefixTreeFilter ex
     private Cell lastLeaf = null;//just for assertion
 
     private SmallDocSet getLeafDocs(Cell leafCell, Bits acceptContains) throws IOException {
-      assert new BytesRef(leafCell.getTokenBytes()).equals(termBytes);
+      assert leafCell.getTokenBytesNoLeaf(null).equals(termBytes);
       assert ! leafCell.equals(lastLeaf);//don't call for same leaf again
       lastLeaf = leafCell;
 

Modified: lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java (original)
+++ lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java Sat Apr  5 09:02:57 2014
@@ -46,7 +46,7 @@ public class PointPrefixTreeFieldCachePr
   protected Point readShape(BytesRef term) {
     scanCell = grid.getCell(term.bytes, term.offset, term.length, scanCell);
     if (scanCell.getLevel() == grid.getMaxLevels() && !scanCell.isLeaf())
-      return scanCell.getCenter();
+      return scanCell.getShape().getCenter();
     return null;
   }
 }

Modified: lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java (original)
+++ lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java Sat Apr  5 09:02:57 2014
@@ -19,8 +19,6 @@ package org.apache.lucene.spatial.prefix
 
 import com.spatial4j.core.shape.Point;
 import com.spatial4j.core.shape.Shape;
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.index.FieldInfo;
@@ -31,7 +29,6 @@ import org.apache.lucene.spatial.prefix.
 import org.apache.lucene.spatial.query.SpatialArgs;
 import org.apache.lucene.spatial.util.ShapeFieldCacheDistanceValueSource;
 
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -125,13 +122,12 @@ public abstract class PrefixTreeStrategy
 
   public Field[] createIndexableFields(Shape shape, double distErr) {
     int detailLevel = grid.getLevelForDistance(distErr);
+    // note: maybe CellTokenStream should do this line, but it doesn't matter and it would create extra
+    // coupling
     List<Cell> cells = grid.getCells(shape, detailLevel, true, simplifyIndexedCells);//intermediates cells
 
-    //TODO is CellTokenStream supposed to be re-used somehow? see Uwe's comments:
-    //  http://code.google.com/p/lucene-spatial-playground/issues/detail?id=4
-
     Field field = new Field(getFieldName(),
-        new CellTokenStream(cells.iterator()), FIELD_TYPE);
+        new CellTokenStream().setCells(cells.iterator()), FIELD_TYPE);
     return new Field[]{field};
   }
 
@@ -146,41 +142,6 @@ public abstract class PrefixTreeStrategy
     FIELD_TYPE.freeze();
   }
 
-  /** Outputs the tokenString of a cell, and if its a leaf, outputs it again with the leaf byte. */
-  final static class CellTokenStream extends TokenStream {
-
-    private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
-
-    private Iterator<Cell> iter = null;
-
-    public CellTokenStream(Iterator<Cell> tokens) {
-      this.iter = tokens;
-    }
-
-    CharSequence nextTokenStringNeedingLeaf = null;
-
-    @Override
-    public boolean incrementToken() {
-      clearAttributes();
-      if (nextTokenStringNeedingLeaf != null) {
-        termAtt.append(nextTokenStringNeedingLeaf);
-        termAtt.append((char) Cell.LEAF_BYTE);
-        nextTokenStringNeedingLeaf = null;
-        return true;
-      }
-      if (iter.hasNext()) {
-        Cell cell = iter.next();
-        CharSequence token = cell.getTokenString();
-        termAtt.append(token);
-        if (cell.isLeaf())
-          nextTokenStringNeedingLeaf = token;
-        return true;
-      }
-      return false;
-    }
-
-  }
-
   @Override
   public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
     PointPrefixTreeFieldCacheProvider p = provider.get( getFieldName() );

Modified: lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java (original)
+++ lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java Sat Apr  5 09:02:57 2014
@@ -61,7 +61,7 @@ public class TermQueryPrefixTreeStrategy
     BytesRef[] terms = new BytesRef[cells.size()];
     int i = 0;
     for (Cell cell : cells) {
-      terms[i++] = new BytesRef(cell.getTokenString());//TODO use cell.getTokenBytes()
+      terms[i++] = cell.getTokenBytesNoLeaf(null);
     }
     return new TermsFilter(getFieldName(), terms);
   }

Modified: lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java (original)
+++ lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java Sat Apr  5 09:02:57 2014
@@ -20,6 +20,8 @@ package org.apache.lucene.spatial.prefix
 import com.spatial4j.core.shape.Point;
 import com.spatial4j.core.shape.Shape;
 import com.spatial4j.core.shape.SpatialRelation;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.StringHelper;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -27,74 +29,45 @@ import java.util.Collections;
 import java.util.List;
 
 /**
- * Represents a grid cell. These are not necessarily thread-safe, although new
- * Cell("") (world cell) must be.
+ * Represents a grid cell. These are not necessarily thread-safe, although calling {@link #getShape()} will
+ * sufficiently prepare it to be so, if needed.
  *
  * @lucene.experimental
  */
-public abstract class Cell implements Comparable<Cell> {
-  public static final byte LEAF_BYTE = '+';//NOTE: must sort before letters & numbers
+public abstract class Cell {
 
-  /*
-  Holds a byte[] and/or String representation of the cell. Both are lazy constructed from the other.
-  Neither contains the trailing leaf byte.
-   */
+  private static final byte LEAF_BYTE = '+';//NOTE: must sort before letters & numbers
+
+  //Arguably we could simply use a BytesRef, using an extra Object.
   private byte[] bytes;
   private int b_off;
   private int b_len;
 
-  private String token;//this is the only part of equality
-
   /**
    * When set via getSubCells(filter), it is the relationship between this cell
-   * and the given shape filter.
+   * and the given shape filter. Doesn't participate in shape equality.
    */
   protected SpatialRelation shapeRel;
 
-  /**
-   * Always false for points. Otherwise, indicate no further sub-cells are going
-   * to be provided because shapeRel is WITHIN or maxLevels or a detailLevel is
-   * hit.
-   */
-  protected boolean leaf;
-
-  protected Cell(String token) {
-    this.token = token;
-    if (token.length() > 0 && token.charAt(token.length() - 1) == (char) LEAF_BYTE) {
-      this.token = token.substring(0, token.length() - 1);
-      setLeaf();
-    }
-
-    if (getLevel() == 0)
-      getShape();//ensure any lazy instantiation completes to make this threadsafe
-  }
-
+  /** Warning: Refers to the same bytes (no copy). If {@link #setLeaf()} is subsequently called then it
+   * may modify bytes. */
   protected Cell(byte[] bytes, int off, int len) {
     this.bytes = bytes;
     this.b_off = off;
     this.b_len = len;
-    b_fixLeaf();
   }
 
+  /** Warning: Refers to the same bytes (no copy). If {@link #setLeaf()} is subsequently called then it
+   * may modify bytes. */
   public void reset(byte[] bytes, int off, int len) {
     assert getLevel() != 0;
-    token = null;
     shapeRel = null;
     this.bytes = bytes;
     this.b_off = off;
     this.b_len = len;
-    b_fixLeaf();
   }
 
-  private void b_fixLeaf() {
-    //note that non-point shapes always have the maxLevels cell set with setLeaf
-    if (bytes[b_off + b_len - 1] == LEAF_BYTE) {
-      b_len--;
-      setLeaf();
-    } else {
-      leaf = false;
-    }
-  }
+  protected abstract SpatialPrefixTree getGrid();
 
   public SpatialRelation getShapeRel() {
     return shapeRel;
@@ -105,47 +78,68 @@ public abstract class Cell implements Co
    * further cells with this prefix for the shape (always true at maxLevels).
    */
   public boolean isLeaf() {
-    return leaf;
+    return (b_len > 0 && bytes[b_off + b_len - 1] == LEAF_BYTE);
   }
 
-  /** Note: not supported at level 0. */
+  /** Modifies the bytes to reflect that this is a leaf. Warning: never invoke from a cell
+   * initialized to reference the same bytes from termsEnum, which should be treated as immutable.
+   * Note: not supported at level 0. */
   public void setLeaf() {
     assert getLevel() != 0;
-    leaf = true;
+    if (isLeaf())
+      return;
+    //if isn't big enough, we have to copy
+    if (bytes.length < b_off + b_len) {
+      //hopefully this copying doesn't happen too much (DWS: I checked and it doesn't seem to happen)
+      byte[] copy = new byte[b_len + 1];
+      System.arraycopy(bytes, b_off, copy, 0, b_len);
+      copy[b_len++] = LEAF_BYTE;
+      bytes = copy;
+      b_off = 0;
+    } else {
+      bytes[b_off + b_len++] = LEAF_BYTE;
+    }
   }
 
   /**
-   * Note: doesn't contain a trailing leaf byte.
+   * Returns the bytes for this cell.
+   * The result param is used to save object allocation, though it's bytes aren't used.
+   * @param result where the result goes, or null to create new
    */
-  public String getTokenString() {
-    if (token == null) {
-      token = new String(bytes, b_off, b_len, SpatialPrefixTree.UTF8);
-    }
-    return token;
+  public BytesRef getTokenBytes(BytesRef result) {
+    if (result == null)
+      result = new BytesRef();
+    result.bytes = bytes;
+    result.offset = b_off;
+    result.length = b_len;
+    return result;
   }
 
   /**
-   * Note: doesn't contain a trailing leaf byte.
+   * Returns the bytes for this cell, without leaf set. The bytes should sort before any
+   * cells that have the leaf set for the spatial location.
+   * The result param is used to save object allocation, though it's bytes aren't used.
+   * @param result where the result goes, or null to create new
    */
-  public byte[] getTokenBytes() {
-    if (bytes != null) {
-      if (b_off != 0 || b_len != bytes.length) {
-        throw new IllegalStateException("Not supported if byte[] needs to be recreated.");
-      }
-    } else {
-      bytes = token.getBytes(SpatialPrefixTree.UTF8);
-      b_off = 0;
-      b_len = bytes.length;
-    }
-    return bytes;
+  public BytesRef getTokenBytesNoLeaf(BytesRef result) {
+    result = getTokenBytes(result);
+    if (isLeaf())
+      result.length--;
+    return result;
   }
 
+  /** Level 0 is the world (and has no parent), from then on a higher level means a smaller
+   * cell than the level before it.
+   */
   public int getLevel() {
-    return token != null ? token.length() : b_len;
+    return isLeaf() ? b_len - 1 : b_len;
   }
 
-  //TODO add getParent() and update some algorithms to use this?
-  //public Cell getParent();
+  /** Gets the parent cell that contains this one. Don't call on the world cell. */
+  public Cell getParent() {
+    assert getLevel() > 0;
+    return getGrid().getCell(bytes, b_off, b_len - (isLeaf() ? 2 : 1));
+  }
 
   /**
    * Like {@link #getSubCells()} but with the results filtered by a shape. If
@@ -196,8 +190,6 @@ public abstract class Cell implements Co
    */
   public abstract Cell getSubCell(Point p);
 
-  //TODO Cell getSubCell(byte b)
-
   /**
    * Gets the cells at the next grid cell level that cover this cell.
    * Precondition: Never called when getLevel() == maxLevel.
@@ -211,30 +203,45 @@ public abstract class Cell implements Co
    */
   public abstract int getSubCellsSize();
 
+  /** Gets the shape for this cell; typically a Rectangle. This method also serves to trigger any lazy
+   * loading needed to make the cell instance thread-safe.
+   */
   public abstract Shape getShape();
 
+  /** TODO remove once no longer used. */
   public Point getCenter() {
     return getShape().getCenter();
   }
 
   @Override
-  public int compareTo(Cell o) {
-    return getTokenString().compareTo(o.getTokenString());
-  }
-
-  @Override
   public boolean equals(Object obj) {
-    return !(obj == null || !(obj instanceof Cell)) && getTokenString().equals(((Cell) obj).getTokenString());
+    //this method isn't "normally" called; just in asserts/tests
+    if (obj instanceof Cell) {
+      Cell cell = (Cell) obj;
+      return getTokenBytes(null).equals(cell.getTokenBytes(null));
+    } else {
+      return false;
+    }
   }
 
   @Override
   public int hashCode() {
-    return getTokenString().hashCode();
+    return getTokenBytesNoLeaf(null).hashCode();
   }
 
   @Override
   public String toString() {
-    return getTokenString() + (isLeaf() ? (char) LEAF_BYTE : "");
+    //this method isn't "normally" called; just in asserts/tests
+    return getTokenBytes(null).utf8ToString();
   }
 
+  /**
+   * Returns if the target term is within/underneath this cell; not necessarily a direct descendant.
+   * @param bytesNoLeaf must be getTokenBytesNoLeaf
+   * @param term the term
+   */
+  public boolean isWithin(BytesRef bytesNoLeaf, BytesRef term) {
+    assert bytesNoLeaf.equals(getTokenBytesNoLeaf(null));
+    return StringHelper.startsWith(term, bytesNoLeaf);
+  }
 }

Modified: lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java (original)
+++ lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java Sat Apr  5 09:02:57 2014
@@ -84,18 +84,29 @@ public class GeohashPrefixTree extends S
   }
 
   @Override
-  public Cell getCell(String token) {
-    return new GhCell(token);
-  }
-
-  @Override
   public Cell getCell(byte[] bytes, int offset, int len) {
     return new GhCell(bytes, offset, len);
   }
 
+  private static byte[] stringToBytesPlus1(String token) {
+    //copy ASCII token to byte array with one extra spot for eventual LEAF_BYTE if needed
+    byte[] bytes = new byte[token.length() + 1];
+    for (int i = 0; i < token.length(); i++) {
+      bytes[i] = (byte) token.charAt(i);
+    }
+    return bytes;
+  }
+
   class GhCell extends Cell {
-    GhCell(String token) {
-      super(token);
+
+    private Shape shape;//cache
+    private String geohash;//cache; never has leaf byte, simply a geohash
+
+    GhCell(String geohash) {
+      super(stringToBytesPlus1(geohash), 0, geohash.length());
+      this.geohash = geohash;
+      if (isLeaf())
+        this.geohash = geohash.substring(0, geohash.length() - 1);
     }
 
     GhCell(byte[] bytes, int off, int len) {
@@ -103,8 +114,12 @@ public class GeohashPrefixTree extends S
     }
 
     @Override
+    protected SpatialPrefixTree getGrid() { return GeohashPrefixTree.this; }
+
+    @Override
     public void reset(byte[] bytes, int off, int len) {
       super.reset(bytes, off, len);
+      geohash = null;
       shape = null;
     }
 
@@ -125,26 +140,26 @@ public class GeohashPrefixTree extends S
 
     @Override
     public Cell getSubCell(Point p) {
-      return GeohashPrefixTree.this.getCell(p, getLevel() + 1);//not performant!
+      return getGrid().getCell(p, getLevel() + 1);//not performant!
     }
 
-    private Shape shape;//cache
-
     @Override
     public Shape getShape() {
       if (shape == null) {
-        shape = GeohashUtils.decodeBoundary(getGeohash(), ctx);
+        shape = GeohashUtils.decodeBoundary(getGeohash(), getGrid().getSpatialContext());
       }
       return shape;
     }
 
     @Override
     public Point getCenter() {
-      return GeohashUtils.decode(getGeohash(), ctx);
+      return GeohashUtils.decode(getGeohash(), getGrid().getSpatialContext());
     }
 
     private String getGeohash() {
-      return getTokenString();
+      if (geohash == null)
+        geohash = getTokenBytesNoLeaf(null).utf8ToString();
+      return geohash;
     }
 
   }//class GhCell

Modified: lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java (original)
+++ lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java Sat Apr  5 09:02:57 2014
@@ -22,6 +22,7 @@ import com.spatial4j.core.shape.Point;
 import com.spatial4j.core.shape.Rectangle;
 import com.spatial4j.core.shape.Shape;
 import com.spatial4j.core.shape.SpatialRelation;
+import org.apache.lucene.util.BytesRef;
 
 import java.io.PrintStream;
 import java.text.NumberFormat;
@@ -142,16 +143,11 @@ public class QuadPrefixTree extends Spat
   @Override
   public Cell getCell(Point p, int level) {
     List<Cell> cells = new ArrayList<>(1);
-    build(xmid, ymid, 0, cells, new StringBuilder(), ctx.makePoint(p.getX(),p.getY()), level);
+    build(xmid, ymid, 0, cells, new BytesRef(maxLevels+1), ctx.makePoint(p.getX(),p.getY()), level);
     return cells.get(0);//note cells could be longer if p on edge
   }
 
   @Override
-  public Cell getCell(String token) {
-    return new QuadCell(token);
-  }
-
-  @Override
   public Cell getCell(byte[] bytes, int offset, int len) {
     return new QuadCell(bytes, offset, len);
   }
@@ -161,10 +157,10 @@ public class QuadPrefixTree extends Spat
       double y,
       int level,
       List<Cell> matches,
-      StringBuilder str,
+      BytesRef str,
       Shape shape,
       int maxLevel) {
-    assert str.length() == level;
+    assert str.length == level;
     double w = levelW[level] / 2;
     double h = levelH[level] / 2;
 
@@ -187,50 +183,50 @@ public class QuadPrefixTree extends Spat
       double cy,
       int level,
       List<Cell> matches,
-      StringBuilder str,
+      BytesRef str,
       Shape shape,
       int maxLevel) {
-    assert str.length() == level;
+    assert str.length == level;
+    assert str.offset == 0;
     double w = levelW[level] / 2;
     double h = levelH[level] / 2;
 
-    int strlen = str.length();
+    int strlen = str.length;
     Rectangle rectangle = ctx.makeRectangle(cx - w, cx + w, cy - h, cy + h);
     SpatialRelation v = shape.relate(rectangle);
     if (SpatialRelation.CONTAINS == v) {
-      str.append(c);
+      str.bytes[str.length++] = (byte)c;//append
       //str.append(SpatialPrefixGrid.COVER);
-      matches.add(new QuadCell(str.toString(),v.transpose()));
+      matches.add(new QuadCell(BytesRef.deepCopyOf(str), v.transpose()));
     } else if (SpatialRelation.DISJOINT == v) {
       // nothing
     } else { // SpatialRelation.WITHIN, SpatialRelation.INTERSECTS
-      str.append(c);
+      str.bytes[str.length++] = (byte)c;//append
 
       int nextLevel = level+1;
       if (nextLevel >= maxLevel) {
         //str.append(SpatialPrefixGrid.INTERSECTS);
-        matches.add(new QuadCell(str.toString(),v.transpose()));
+        matches.add(new QuadCell(BytesRef.deepCopyOf(str), v.transpose()));
       } else {
         build(cx, cy, nextLevel, matches, str, shape, maxLevel);
       }
     }
-    str.setLength(strlen);
+    str.length = strlen;
   }
 
-  class QuadCell extends Cell {
+  class QuadCell extends Cell{
 
-    public QuadCell(String token) {
-      super(token);
+    QuadCell(byte[] bytes, int off, int len) {
+      super(bytes, off, len);
     }
 
-    public QuadCell(String token, SpatialRelation shapeRel) {
-      super(token);
+    QuadCell(BytesRef str, SpatialRelation shapeRel) {
+      this(str.bytes, str.offset, str.length);
       this.shapeRel = shapeRel;
     }
 
-    QuadCell(byte[] bytes, int off, int len) {
-      super(bytes, off, len);
-    }
+    @Override
+    protected SpatialPrefixTree getGrid() { return QuadPrefixTree.this; }
 
     @Override
     public void reset(byte[] bytes, int off, int len) {
@@ -240,14 +236,26 @@ public class QuadPrefixTree extends Spat
 
     @Override
     public Collection<Cell> getSubCells() {
+      BytesRef source = getTokenBytesNoLeaf(null);
+      BytesRef target = new BytesRef();
+
       List<Cell> cells = new ArrayList<>(4);
-      cells.add(new QuadCell(getTokenString()+"A"));
-      cells.add(new QuadCell(getTokenString()+"B"));
-      cells.add(new QuadCell(getTokenString()+"C"));
-      cells.add(new QuadCell(getTokenString()+"D"));
+      cells.add(new QuadCell(concat(source, (byte)'A', target), null));
+      cells.add(new QuadCell(concat(source, (byte)'B', target), null));
+      cells.add(new QuadCell(concat(source, (byte)'C', target), null));
+      cells.add(new QuadCell(concat(source, (byte)'D', target), null));
       return cells;
     }
 
+    private BytesRef concat(BytesRef source, byte b, BytesRef target) {
+      assert target.offset == 0;
+      target.bytes = new byte[source.length + 2];//+2 for new char + potential leaf
+      target.length = 0;
+      target.copyBytes(source);
+      target.bytes[target.length++] = b;
+      return target;
+    }
+
     @Override
     public int getSubCellsSize() {
       return 4;
@@ -268,27 +276,30 @@ public class QuadPrefixTree extends Spat
     }
 
     private Rectangle makeShape() {
-      String token = getTokenString();
+      BytesRef token = getTokenBytesNoLeaf(null);
       double xmin = QuadPrefixTree.this.xmin;
       double ymin = QuadPrefixTree.this.ymin;
 
-      for (int i = 0; i < token.length(); i++) {
-        char c = token.charAt(i);
-        if ('A' == c || 'a' == c) {
-          ymin += levelH[i];
-        } else if ('B' == c || 'b' == c) {
-          xmin += levelW[i];
-          ymin += levelH[i];
-        } else if ('C' == c || 'c' == c) {
-          // nothing really
-        }
-        else if('D' == c || 'd' == c) {
-          xmin += levelW[i];
-        } else {
-          throw new RuntimeException("unexpected char: " + c);
+      for (int i = 0; i < token.length; i++) {
+        byte c = token.bytes[token.offset + i];
+        switch (c) {
+          case 'A':
+            ymin += levelH[i];
+            break;
+          case 'B':
+            xmin += levelW[i];
+            ymin += levelH[i];
+            break;
+          case 'C':
+            break;//nothing really
+          case 'D':
+            xmin += levelW[i];
+            break;
+          default:
+            throw new RuntimeException("unexpected char: " + c);
         }
       }
-      int len = token.length();
+      int len = token.length;
       double width, height;
       if (len > 0) {
         width = levelW[len-1];

Modified: lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java (original)
+++ lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java Sat Apr  5 09:02:57 2014
@@ -21,10 +21,12 @@ import com.spatial4j.core.context.Spatia
 import com.spatial4j.core.shape.Point;
 import com.spatial4j.core.shape.Rectangle;
 import com.spatial4j.core.shape.Shape;
+import org.apache.lucene.util.BytesRef;
 
 import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
@@ -103,14 +105,14 @@ public abstract class SpatialPrefixTree 
   private transient Cell worldCell;//cached
 
   /**
-   * Returns the level 0 cell which encompasses all spatial data. Equivalent to {@link #getCell(String)} with "".
-   * This cell is threadsafe, just like a spatial prefix grid is, although cells aren't
-   * generally threadsafe.
-   * TODO rename to getTopCell or is this fine?
+   * Returns the level 0 cell which encompasses all spatial data. Equivalent to {@link #getCell(byte[], int, int)} with
+   * no bytes. This cell is thread-safe, just like a spatial prefix grid is, although cells aren't
+   * generally thread-safe.
    */
-  public Cell getWorldCell() {
+  public Cell getWorldCell() {//another possible name: getTopCell
     if (worldCell == null) {
-      worldCell = getCell("");
+      worldCell = getCell(BytesRef.EMPTY_BYTES, 0, 0);
+      worldCell.getShape();//lazy load; make thread-safe
     }
     return worldCell;
   }
@@ -119,8 +121,6 @@ public abstract class SpatialPrefixTree 
    * The cell for the specified token. The empty string should be equal to {@link #getWorldCell()}.
    * Precondition: Never called when token length > maxLevel.
    */
-  public abstract Cell getCell(String token);
-
   public abstract Cell getCell(byte[] bytes, int offset, int len);
 
   public final Cell getCell(byte[] bytes, int offset, int len, Cell target) {
@@ -215,40 +215,23 @@ public abstract class SpatialPrefixTree 
    * A Point-optimized implementation of
    * {@link #getCells(com.spatial4j.core.shape.Shape, int, boolean, boolean)}. That
    * method in facts calls this for points.
-   * <p/>
-   * This implementation depends on {@link #getCell(String)} being fast, as its
-   * called repeatedly when incPlarents is true.
    */
   public List<Cell> getCells(Point p, int detailLevel, boolean inclParents) {
     Cell cell = getCell(p, detailLevel);
-    if (!inclParents) {
+    assert !cell.isLeaf();
+    if (!inclParents || detailLevel == 1) {
       return Collections.singletonList(cell);
     }
 
-    String endToken = cell.getTokenString();
-    assert endToken.length() == detailLevel;
-    List<Cell> cells = new ArrayList<>(detailLevel);
-    for (int i = 1; i < detailLevel; i++) {
-      cells.add(getCell(endToken.substring(0, i)));//TODO refactor: add a cell.getParent()
+    //fill in reverse order to be sorted
+    Cell[] cells = new Cell[detailLevel];
+    for (int i = detailLevel-1; true; i--) {
+      cells[i] = cell;
+      if (i == 0)
+        break;
+      cell = cell.getParent();
     }
-    cells.add(cell);
-    return cells;
+    return Arrays.asList(cells);
   }
 
-  /**
-   * Will add the trailing leaf byte for leaves. This isn't particularly efficient.
-   * @deprecated TODO remove; not used and not interesting, don't need collection in & out
-   */
-  public static List<String> cellsToTokenStrings(Collection<Cell> cells) {
-    List<String> tokens = new ArrayList<>((cells.size()));
-    for (Cell cell : cells) {
-      final String token = cell.getTokenString();
-      if (cell.isLeaf()) {
-        tokens.add(token + (char) Cell.LEAF_BYTE);
-      } else {
-        tokens.add(token);
-      }
-    }
-    return tokens;
-  }
 }

Modified: lucene/dev/branches/solr5914/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeTest.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeTest.java (original)
+++ lucene/dev/branches/solr5914/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeTest.java Sat Apr  5 09:02:57 2014
@@ -35,6 +35,9 @@ import org.apache.lucene.spatial.query.S
 import org.junit.Before;
 import org.junit.Test;
 
+import java.util.ArrayList;
+import java.util.List;
+
 public class SpatialPrefixTreeTest extends SpatialTestCase {
 
   //TODO plug in others and test them
@@ -56,9 +59,10 @@ public class SpatialPrefixTreeTest exten
     Cell c = trie.getWorldCell();
     assertEquals(0, c.getLevel());
     assertEquals(ctx.getWorldBounds(), c.getShape());
-    while(c.getLevel() < trie.getMaxLevels()) {
+    while (c.getLevel() < trie.getMaxLevels()) {
       prevC = c;
-      c = c.getSubCells().iterator().next();//TODO random which one?
+      List<Cell> subCells = new ArrayList<>(c.getSubCells());
+      c = subCells.get(random().nextInt(subCells.size()-1));
       
       assertEquals(prevC.getLevel()+1,c.getLevel());
       Rectangle prevNShape = (Rectangle) prevC.getShape();

Modified: lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkOutOfOrderScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkOutOfOrderScorer.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkOutOfOrderScorer.java (original)
+++ lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkOutOfOrderScorer.java Sat Apr  5 09:02:57 2014
@@ -18,93 +18,37 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.lang.ref.WeakReference;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
 import java.util.Random;
-import java.util.WeakHashMap;
 
-import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.util.VirtualMethod;
-
-/** A crazy {@link BulkScorer} that wraps a {@link Scorer}
+/** A crazy {@link BulkScorer} that wraps another {@link BulkScorer}
  *  but shuffles the order of the collected documents. */
 public class AssertingBulkOutOfOrderScorer extends BulkScorer {
 
+  final BulkScorer in;
   final Random random;
-  final Scorer scorer;
 
-  public AssertingBulkOutOfOrderScorer(Random random, Scorer scorer) {
+  public AssertingBulkOutOfOrderScorer(Random random, BulkScorer in) {
+    this.in = in;
     this.random = random;
-    this.scorer = scorer;
-  }
-
-  private void shuffle(int[] docIDs, float[] scores, int[] freqs, int size) {
-    for (int i = size - 1; i > 0; --i) {
-      final int other = random.nextInt(i + 1);
-
-      final int tmpDoc = docIDs[i];
-      docIDs[i] = docIDs[other];
-      docIDs[other] = tmpDoc;
-
-      final float tmpScore = scores[i];
-      scores[i] = scores[other];
-      scores[other] = tmpScore;
-      
-      final int tmpFreq = freqs[i];
-      freqs[i] = freqs[other];
-      freqs[other] = tmpFreq;
-    }
   }
 
-  private static void flush(int[] docIDs, float[] scores, int[] freqs, int size,
-      FakeScorer scorer, Collector collector) throws IOException {
-    for (int i = 0; i < size; ++i) {
-      scorer.doc = docIDs[i];
-      scorer.freq = freqs[i];
-      scorer.score = scores[i];
-      collector.collect(scorer.doc);
-    }
+  @Override
+  public boolean score(LeafCollector collector, int max) throws IOException {
+    final RandomOrderCollector randomCollector = new RandomOrderCollector(random, collector);
+    final boolean remaining = in.score(randomCollector, max);
+    randomCollector.flush();
+    return remaining;
   }
 
   @Override
-  public boolean score(Collector collector, int max) throws IOException {
-    if (scorer.docID() == -1) {
-      scorer.nextDoc();
-    }
-
-    FakeScorer fake = new FakeScorer();
-    collector.setScorer(fake);
-
-    final int bufferSize = 1 + random.nextInt(100);
-    final int[] docIDs = new int[bufferSize];
-    final float[] scores = new float[bufferSize];
-    final int[] freqs = new int[bufferSize];
-
-    int buffered = 0;
-    int doc = scorer.docID();
-    while (doc < max) {
-      docIDs[buffered] = doc;
-      scores[buffered] = scorer.score();
-      freqs[buffered] = scorer.freq();
-
-      if (++buffered == bufferSize) {
-        shuffle(docIDs, scores, freqs, buffered);
-        flush(docIDs, scores, freqs, buffered, fake, collector);
-        buffered = 0;
-      }
-      doc = scorer.nextDoc();
-    }
-
-    shuffle(docIDs, scores, freqs, buffered);
-    flush(docIDs, scores, freqs, buffered, fake, collector);
-
-    return doc != Scorer.NO_MORE_DOCS;
+  public void score(LeafCollector collector) throws IOException {
+    final RandomOrderCollector randomCollector = new RandomOrderCollector(random, collector);
+    in.score(randomCollector);
+    randomCollector.flush();
   }
 
   @Override
   public String toString() {
-    return "AssertingBulkOutOfOrderScorer(" + scorer + ")";
+    return "AssertingBulkOutOfOrderScorer(" + in + ")";
   }
 }

Modified: lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkScorer.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkScorer.java (original)
+++ lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkScorer.java Sat Apr  5 09:02:57 2014
@@ -31,8 +31,8 @@ import org.apache.lucene.util.VirtualMet
 /** Wraps a Scorer with additional checks */
 public class AssertingBulkScorer extends BulkScorer {
 
-  private static final VirtualMethod<BulkScorer> SCORE_COLLECTOR = new VirtualMethod<BulkScorer>(BulkScorer.class, "score", Collector.class);
-  private static final VirtualMethod<BulkScorer> SCORE_COLLECTOR_RANGE = new VirtualMethod<BulkScorer>(BulkScorer.class, "score", Collector.class, int.class);
+  private static final VirtualMethod<BulkScorer> SCORE_COLLECTOR = new VirtualMethod<BulkScorer>(BulkScorer.class, "score", LeafCollector.class);
+  private static final VirtualMethod<BulkScorer> SCORE_COLLECTOR_RANGE = new VirtualMethod<BulkScorer>(BulkScorer.class, "score", LeafCollector.class, int.class);
 
   public static BulkScorer wrap(Random random, BulkScorer other) {
     if (other == null || other instanceof AssertingBulkScorer) {
@@ -58,7 +58,7 @@ public class AssertingBulkScorer extends
   }
 
   @Override
-  public void score(Collector collector) throws IOException {
+  public void score(LeafCollector collector) throws IOException {
     if (random.nextBoolean()) {
       try {
         final boolean remaining = in.score(collector, DocsEnum.NO_MORE_DOCS);
@@ -72,7 +72,7 @@ public class AssertingBulkScorer extends
   }
 
   @Override
-  public boolean score(Collector collector, int max) throws IOException {
+  public boolean score(LeafCollector collector, int max) throws IOException {
     return in.score(collector, max);
   }
 

Modified: lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/AssertingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/AssertingCollector.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/AssertingCollector.java (original)
+++ lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/AssertingCollector.java Sat Apr  5 09:02:57 2014
@@ -25,46 +25,42 @@ import org.apache.lucene.index.AtomicRea
 /** Wraps another Collector and checks that
  *  acceptsDocsOutOfOrder is respected. */
 
-public class AssertingCollector extends Collector {
+public class AssertingCollector extends FilterCollector {
 
   public static Collector wrap(Random random, Collector other, boolean inOrder) {
     return other instanceof AssertingCollector ? other : new AssertingCollector(random, other, inOrder);
   }
 
   final Random random;
-  final Collector in;
   final boolean inOrder;
-  int lastCollected;
 
   AssertingCollector(Random random, Collector in, boolean inOrder) {
+    super(in);
     this.random = random;
-    this.in = in;
     this.inOrder = inOrder;
-    lastCollected = -1;
   }
 
   @Override
-  public void setScorer(Scorer scorer) throws IOException {
-    in.setScorer(AssertingScorer.getAssertingScorer(random, scorer));
-  }
+  public LeafCollector getLeafCollector(AtomicReaderContext context) throws IOException {
+    return new FilterLeafCollector(super.getLeafCollector(context)) {
 
-  @Override
-  public void collect(int doc) throws IOException {
-    if (inOrder || !acceptsDocsOutOfOrder()) {
-      assert doc > lastCollected : "Out of order : " + lastCollected + " " + doc;
-    }
-    in.collect(doc);
-    lastCollected = doc;
-  }
+      int lastCollected = -1;
 
-  @Override
-  public void setNextReader(AtomicReaderContext context) throws IOException {
-    lastCollected = -1;
-  }
+      @Override
+      public void setScorer(Scorer scorer) throws IOException {
+        super.setScorer(AssertingScorer.getAssertingScorer(random, scorer));
+      }
+
+      @Override
+      public void collect(int doc) throws IOException {
+        if (inOrder || !acceptsDocsOutOfOrder()) {
+          assert doc > lastCollected : "Out of order : " + lastCollected + " " + doc;
+        }
+        in.collect(doc);
+        lastCollected = doc;
+      }
 
-  @Override
-  public boolean acceptsDocsOutOfOrder() {
-    return in.acceptsDocsOutOfOrder();
+    };
   }
 
 }

Modified: lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java (original)
+++ lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java Sat Apr  5 09:02:57 2014
@@ -79,22 +79,20 @@ class AssertingWeight extends Weight {
     if (AssertingBulkScorer.shouldWrap(inScorer)) {
       // The incoming scorer already has a specialized
       // implementation for BulkScorer, so we should use it:
-      return AssertingBulkScorer.wrap(new Random(random.nextLong()), inScorer);
-    } else if (scoreDocsInOrder == false && random.nextBoolean()) {
+      inScorer = AssertingBulkScorer.wrap(new Random(random.nextLong()), inScorer);
+    } else if (random.nextBoolean()) {
+      // Let super wrap this.scorer instead, so we use
+      // AssertingScorer:
+      inScorer = super.bulkScorer(context, scoreDocsInOrder, acceptDocs);
+    }
+
+    if (scoreDocsInOrder == false && random.nextBoolean()) {
       // The caller claims it can handle out-of-order
       // docs; let's confirm that by pulling docs and
       // randomly shuffling them before collection:
-      //Scorer scorer = in.scorer(context, acceptDocs);
-      Scorer scorer = scorer(context, acceptDocs);
-
-      // Scorer should not be null if bulkScorer wasn't:
-      assert scorer != null;
-      return new AssertingBulkOutOfOrderScorer(new Random(random.nextLong()), scorer);
-    } else {
-      // Let super wrap this.scorer instead, so we use
-      // AssertingScorer:
-      return super.bulkScorer(context, scoreDocsInOrder, acceptDocs);
+      inScorer = new AssertingBulkOutOfOrderScorer(new Random(random.nextLong()), inScorer);
     }
+    return inScorer;
   }
 
   @Override

Modified: lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java (original)
+++ lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java Sat Apr  5 09:02:57 2014
@@ -123,7 +123,7 @@ public class CheckHits {
   /**
    * Just collects document ids into a set.
    */
-  public static class SetCollector extends Collector {
+  public static class SetCollector extends SimpleCollector {
     final Set<Integer> bag;
     public SetCollector(Set<Integer> bag) {
       this.bag = bag;
@@ -136,7 +136,7 @@ public class CheckHits {
       bag.add(Integer.valueOf(doc + base));
     }
     @Override
-    public void setNextReader(AtomicReaderContext context) {
+    protected void doSetNextReader(AtomicReaderContext context) throws IOException {
       base = context.docBase;
     }
     @Override
@@ -464,7 +464,7 @@ public class CheckHits {
    *
    * @see CheckHits#verifyExplanation
    */
-  public static class ExplanationAsserter extends Collector {
+  public static class ExplanationAsserter extends SimpleCollector {
 
     Query q;
     IndexSearcher s;
@@ -508,7 +508,7 @@ public class CheckHits {
                         exp.isMatch());
     }
     @Override
-    public void setNextReader(AtomicReaderContext context) {
+    protected void doSetNextReader(AtomicReaderContext context) throws IOException {
       base = context.docBase;
     }
     @Override

Modified: lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java (original)
+++ lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java Sat Apr  5 09:02:57 2014
@@ -249,7 +249,7 @@ public class QueryUtils {
         final float maxDiff = 1e-5f;
         final AtomicReader lastReader[] = {null};
 
-        s.search(q, new Collector() {
+        s.search(q, new SimpleCollector() {
           private Scorer sc;
           private Scorer scorer;
           private int leafPtr;
@@ -305,7 +305,7 @@ public class QueryUtils {
           }
 
           @Override
-          public void setNextReader(AtomicReaderContext context) throws IOException {
+          protected void doSetNextReader(AtomicReaderContext context) throws IOException {
             // confirm that skipping beyond the last doc, on the
             // previous reader, hits NO_MORE_DOCS
             if (lastReader[0] != null) {
@@ -357,7 +357,7 @@ public class QueryUtils {
     final int lastDoc[] = {-1};
     final AtomicReader lastReader[] = {null};
     final List<AtomicReaderContext> context = s.getTopReaderContext().leaves();
-    s.search(q,new Collector() {
+    s.search(q,new SimpleCollector() {
       private Scorer scorer;
       private int leafPtr;
       private Bits liveDocs;
@@ -392,7 +392,7 @@ public class QueryUtils {
       }
 
       @Override
-      public void setNextReader(AtomicReaderContext context) throws IOException {
+      protected void doSetNextReader(AtomicReaderContext context) throws IOException {
         // confirm that skipping beyond the last doc, on the
         // previous reader, hits NO_MORE_DOCS
         if (lastReader[0] != null) {

Modified: lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java Sat Apr  5 09:02:57 2014
@@ -622,7 +622,8 @@ public class MockDirectoryWrapper extend
     return size;
   }
 
-  private boolean assertNoUnreferencedFilesOnClose = true;
+  // NOTE: This is off by default; see LUCENE-5574
+  private boolean assertNoUnreferencedFilesOnClose;
 
   public void setAssertNoUnrefencedFilesOnClose(boolean v) {
     assertNoUnreferencedFilesOnClose = v;

Modified: lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Sat Apr  5 09:02:57 2014
@@ -872,28 +872,13 @@ public abstract class LuceneTestCase ext
       int maxNumThreadStates = rarely(r) ? TestUtil.nextInt(r, 5, 20) // crazy value
           : TestUtil.nextInt(r, 1, 4); // reasonable value
 
-      Method setIndexerThreadPoolMethod = null;
-      try {
-        // Retrieve the package-private setIndexerThreadPool
-        // method:
-        for(Method m : IndexWriterConfig.class.getDeclaredMethods()) {
-          if (m.getName().equals("setIndexerThreadPool")) {
-            m.setAccessible(true);
-            setIndexerThreadPoolMethod = m;
-            break;
-          }
-        }
-      } catch (Exception e) {
-        // Should not happen?
-        throw new RuntimeException(e);
-      }
-
-      if (setIndexerThreadPoolMethod == null) {
-        throw new RuntimeException("failed to lookup IndexWriterConfig.setIndexerThreadPool method");
-      }
-
       try {
         if (rarely(r)) {
+          // Retrieve the package-private setIndexerThreadPool
+          // method:
+          Method setIndexerThreadPoolMethod = IndexWriterConfig.class.getDeclaredMethod("setIndexerThreadPool",
+            Class.forName("org.apache.lucene.index.DocumentsWriterPerThreadPool"));
+          setIndexerThreadPoolMethod.setAccessible(true);
           Class<?> clazz = Class.forName("org.apache.lucene.index.RandomDocumentsWriterPerThreadPool");
           Constructor<?> ctor = clazz.getConstructor(int.class, Random.class);
           ctor.setAccessible(true);
@@ -904,7 +889,7 @@ public abstract class LuceneTestCase ext
           c.setMaxThreadStates(maxNumThreadStates);
         }
       } catch (Exception e) {
-        throw new RuntimeException(e);
+        Rethrow.rethrow(e);
       }
     }
 
@@ -1097,7 +1082,8 @@ public abstract class LuceneTestCase ext
       }
       return wrapped;
     } catch (Exception e) {
-      throw new RuntimeException(e);
+      Rethrow.rethrow(e);
+      throw null; // dummy to prevent compiler failure
     }
   }
 
@@ -1278,7 +1264,8 @@ public abstract class LuceneTestCase ext
       // try empty ctor
       return clazz.newInstance();
     } catch (Exception e) {
-      throw new RuntimeException(e);
+      Rethrow.rethrow(e);
+      throw null; // dummy to prevent compiler failure
     }
   }
   
@@ -1400,20 +1387,30 @@ public abstract class LuceneTestCase ext
   public static IndexSearcher newSearcher(IndexReader r) {
     return newSearcher(r, true);
   }
-  
+
   /**
    * Create a new searcher over the reader. This searcher might randomly use
-   * threads. if <code>maybeWrap</code> is true, this searcher might wrap the
-   * reader with one that returns null for getSequentialSubReaders.
+   * threads.
    */
   public static IndexSearcher newSearcher(IndexReader r, boolean maybeWrap) {
+    return newSearcher(r, maybeWrap, true);
+  }
+
+  /**
+   * Create a new searcher over the reader. This searcher might randomly use
+   * threads. if <code>maybeWrap</code> is true, this searcher might wrap the
+   * reader with one that returns null for getSequentialSubReaders. If
+   * <code>wrapWithAssertions</code> is true, this searcher might be an
+   * {@link AssertingIndexSearcher} instance.
+   */
+  public static IndexSearcher newSearcher(IndexReader r, boolean maybeWrap, boolean wrapWithAssertions) {
     Random random = random();
     if (usually()) {
       if (maybeWrap) {
         try {
           r = maybeWrapReader(r);
         } catch (IOException e) {
-          throw new AssertionError(e);
+          Rethrow.rethrow(e);
         }
       }
       // TODO: this whole check is a coverage hack, we should move it to tests for various filterreaders.
@@ -1424,10 +1421,15 @@ public abstract class LuceneTestCase ext
         try {
           TestUtil.checkReader(r);
         } catch (IOException e) {
-          throw new AssertionError(e);
+          Rethrow.rethrow(e);
         }
       }
-      IndexSearcher ret = random.nextBoolean() ? new AssertingIndexSearcher(random, r) : new AssertingIndexSearcher(random, r.getContext());
+      final IndexSearcher ret;
+      if (wrapWithAssertions) {
+        ret = random.nextBoolean() ? new AssertingIndexSearcher(random, r) : new AssertingIndexSearcher(random, r.getContext());
+      } else {
+        ret = random.nextBoolean() ? new IndexSearcher(r) : new IndexSearcher(r.getContext());
+      }
       ret.setSimilarity(classEnvRule.similarity);
       return ret;
     } else {
@@ -1454,9 +1456,16 @@ public abstract class LuceneTestCase ext
          }
        });
       }
-      IndexSearcher ret = random.nextBoolean() 
-          ? new AssertingIndexSearcher(random, r, ex)
-          : new AssertingIndexSearcher(random, r.getContext(), ex);
+      IndexSearcher ret;
+      if (wrapWithAssertions) {
+        ret = random.nextBoolean()
+            ? new AssertingIndexSearcher(random, r, ex)
+            : new AssertingIndexSearcher(random, r.getContext(), ex);
+      } else {
+        ret = random.nextBoolean()
+            ? new IndexSearcher(r, ex)
+            : new IndexSearcher(r.getContext(), ex);
+      }
       ret.setSimilarity(classEnvRule.similarity);
       return ret;
     }

Modified: lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/accumulator/BasicAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/accumulator/BasicAccumulator.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/accumulator/BasicAccumulator.java (original)
+++ lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/accumulator/BasicAccumulator.java Sat Apr  5 09:02:57 2014
@@ -80,7 +80,7 @@ public class BasicAccumulator extends Va
   }
   
   @Override
-  public void setNextReader(AtomicReaderContext context) throws IOException {
+  protected void doSetNextReader(AtomicReaderContext context) throws IOException {
     this.context = context;
     for (StatsCollector counter : statsCollectors) {
       counter.setNextReader(context);