You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ha...@apache.org on 2013/08/13 06:06:27 UTC

svn commit: r1513336 [8/11] - in /lucene/dev/branches/lucene3069/lucene: ./ analysis/ analysis/common/ analysis/common/src/java/org/apache/lucene/analysis/charfilter/ analysis/common/src/java/org/apache/lucene/analysis/hunspell/ analysis/common/src/jav...

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesAccumulator.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesAccumulator.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesAccumulator.java Tue Aug 13 04:06:18 2013
@@ -32,19 +32,19 @@ import org.apache.lucene.facet.search.Fa
 import org.apache.lucene.facet.search.FacetResult;
 import org.apache.lucene.facet.search.FacetResultNode;
 import org.apache.lucene.facet.search.FacetsAccumulator;
-import org.apache.lucene.facet.search.FacetsAggregator;
 import org.apache.lucene.facet.search.FacetsCollector.MatchingDocs;
+import org.apache.lucene.facet.search.TaxonomyFacetsAccumulator;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.MultiDocValues.MultiSortedSetDocValues;
 import org.apache.lucene.index.MultiDocValues;
+import org.apache.lucene.index.MultiDocValues.MultiSortedSetDocValues;
 import org.apache.lucene.index.ReaderUtil;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.PriorityQueue;
 
-/** A {@link FacetsAccumulator} that uses previously
+/** A {@link TaxonomyFacetsAccumulator} that uses previously
  *  indexed {@link SortedSetDocValuesFacetFields} to perform faceting,
  *  without require a separate taxonomy index.  Faceting is
  *  a bit slower (~25%), and there is added cost on every
@@ -57,25 +57,34 @@ public class SortedSetDocValuesAccumulat
   final SortedSetDocValuesReaderState state;
   final SortedSetDocValues dv;
   final String field;
-
-  public SortedSetDocValuesAccumulator(FacetSearchParams fsp, SortedSetDocValuesReaderState state) throws IOException {
-    super(fsp, null, null, new FacetArrays(state.getSize()));
+  final FacetArrays facetArrays;
+  
+  /** Constructor with the given facet search params. */
+  public SortedSetDocValuesAccumulator(SortedSetDocValuesReaderState state, FacetSearchParams fsp) 
+      throws IOException {
+    this(state, fsp, null);
+  }
+  
+  public SortedSetDocValuesAccumulator(SortedSetDocValuesReaderState state, FacetSearchParams fsp, FacetArrays arrays) 
+      throws IOException {
+    super(fsp);
     this.state = state;
     this.field = state.getField();
+    this.facetArrays = arrays == null ? new FacetArrays(state.getSize()) : arrays;
     dv = state.getDocValues();
 
     // Check params:
-    for(FacetRequest request : fsp.facetRequests) {
-      if (!(request instanceof CountFacetRequest)) {
-        throw new IllegalArgumentException("this collector only supports CountFacetRequest; got " + request);
+    for (FacetRequest fr : fsp.facetRequests) {
+      if (!(fr instanceof CountFacetRequest)) {
+        throw new IllegalArgumentException("this accumulator only supports CountFacetRequest; got " + fr);
       }
-      if (request.categoryPath.length != 1) {
-        throw new IllegalArgumentException("this collector only supports depth 1 CategoryPath; got " + request.categoryPath);
+      if (fr.categoryPath.length != 1) {
+        throw new IllegalArgumentException("this accumulator only supports 1-level CategoryPath; got " + fr.categoryPath);
       }
-      if (request.getDepth() != 1) {
-        throw new IllegalArgumentException("this collector only supports depth=1; got " + request.getDepth());
+      if (fr.getDepth() != 1) {
+        throw new IllegalArgumentException("this accumulator only supports depth=1; got " + fr.getDepth());
       }
-      String dim = request.categoryPath.components[0];
+      String dim = fr.categoryPath.components[0];
 
       SortedSetDocValuesReaderState.OrdRange ordRange = state.getOrdRange(dim);
       if (ordRange == null) {
@@ -84,131 +93,123 @@ public class SortedSetDocValuesAccumulat
     }
   }
 
-  @Override
-  public FacetsAggregator getAggregator() {
+  /** Keeps highest count results. */
+  static class TopCountPQ extends PriorityQueue<FacetResultNode> {
+    public TopCountPQ(int topN) {
+      super(topN, false);
+    }
+
+    @Override
+    protected boolean lessThan(FacetResultNode a, FacetResultNode b) {
+      if (a.value < b.value) {
+        return true;
+      } else if (a.value > b.value) {
+        return false;
+      } else {
+        return a.ordinal > b.ordinal;
+      }
+    }
+  }
 
-    return new FacetsAggregator() {
+  static class SortedSetAggregator {
 
-      @Override
-      public void aggregate(MatchingDocs matchingDocs, CategoryListParams clp, FacetArrays facetArrays) throws IOException {
+    private final SortedSetDocValuesReaderState state;
+    private final String field;
+    private final SortedSetDocValues dv;
+    
+    public SortedSetAggregator(String field, SortedSetDocValuesReaderState state, SortedSetDocValues dv) {
+      this.field = field;
+      this.state = state;
+      this.dv = dv;
+    }
+    
+    public void aggregate(MatchingDocs matchingDocs, FacetArrays facetArrays) throws IOException {
 
-        AtomicReader reader = matchingDocs.context.reader();
+      AtomicReader reader = matchingDocs.context.reader();
 
-        // LUCENE-5090: make sure the provided reader context "matches"
-        // the top-level reader passed to the
-        // SortedSetDocValuesReaderState, else cryptic
-        // AIOOBE can happen:
-        if (ReaderUtil.getTopLevelContext(matchingDocs.context).reader() != state.origReader) {
-          throw new IllegalStateException("the SortedSetDocValuesReaderState provided to this class does not match the reader being searched; you must create a new SortedSetDocValuesReaderState every time you open a new IndexReader");
-        }
-        
-        SortedSetDocValues segValues = reader.getSortedSetDocValues(field);
-        if (segValues == null) {
-          return;
-        }
+      // LUCENE-5090: make sure the provided reader context "matches"
+      // the top-level reader passed to the
+      // SortedSetDocValuesReaderState, else cryptic
+      // AIOOBE can happen:
+      if (ReaderUtil.getTopLevelContext(matchingDocs.context).reader() != state.origReader) {
+        throw new IllegalStateException("the SortedSetDocValuesReaderState provided to this class does not match the reader being searched; you must create a new SortedSetDocValuesReaderState every time you open a new IndexReader");
+      }
+      
+      SortedSetDocValues segValues = reader.getSortedSetDocValues(field);
+      if (segValues == null) {
+        return;
+      }
 
-        final int[] counts = facetArrays.getIntArray();
-        final int maxDoc = reader.maxDoc();
-        assert maxDoc == matchingDocs.bits.length();
-
-        if (dv instanceof MultiSortedSetDocValues) {
-          MultiDocValues.OrdinalMap ordinalMap = ((MultiSortedSetDocValues) dv).mapping;
-          int segOrd = matchingDocs.context.ord;
-
-          int numSegOrds = (int) segValues.getValueCount();
-
-          if (matchingDocs.totalHits < numSegOrds/10) {
-            // Remap every ord to global ord as we iterate:
-            int doc = 0;
-            while (doc < maxDoc && (doc = matchingDocs.bits.nextSetBit(doc)) != -1) {
-              segValues.setDocument(doc);
-              int term = (int) segValues.nextOrd();
-              while (term != SortedSetDocValues.NO_MORE_ORDS) {
-                counts[(int) ordinalMap.getGlobalOrd(segOrd, term)]++;
-                term = (int) segValues.nextOrd();
-              }
-              ++doc;
-            }
-          } else {
+      final int[] counts = facetArrays.getIntArray();
+      final int maxDoc = reader.maxDoc();
+      assert maxDoc == matchingDocs.bits.length();
 
-            // First count in seg-ord space:
-            final int[] segCounts = new int[numSegOrds];
-            int doc = 0;
-            while (doc < maxDoc && (doc = matchingDocs.bits.nextSetBit(doc)) != -1) {
-              segValues.setDocument(doc);
-              int term = (int) segValues.nextOrd();
-              while (term != SortedSetDocValues.NO_MORE_ORDS) {
-                segCounts[term]++;
-                term = (int) segValues.nextOrd();
-              }
-              ++doc;
-            }
+      if (dv instanceof MultiSortedSetDocValues) {
+        MultiDocValues.OrdinalMap ordinalMap = ((MultiSortedSetDocValues) dv).mapping;
+        int segOrd = matchingDocs.context.ord;
 
-            // Then, migrate to global ords:
-            for(int ord=0;ord<numSegOrds;ord++) {
-              int count = segCounts[ord];
-              if (count != 0) {
-                counts[(int) ordinalMap.getGlobalOrd(segOrd, ord)] += count;
-              }
+        int numSegOrds = (int) segValues.getValueCount();
+
+        if (matchingDocs.totalHits < numSegOrds/10) {
+          // Remap every ord to global ord as we iterate:
+          int doc = 0;
+          while (doc < maxDoc && (doc = matchingDocs.bits.nextSetBit(doc)) != -1) {
+            segValues.setDocument(doc);
+            int term = (int) segValues.nextOrd();
+            while (term != SortedSetDocValues.NO_MORE_ORDS) {
+              counts[(int) ordinalMap.getGlobalOrd(segOrd, term)]++;
+              term = (int) segValues.nextOrd();
             }
+            ++doc;
           }
         } else {
-          // No ord mapping (e.g., single segment index):
-          // just aggregate directly into counts:
 
+          // First count in seg-ord space:
+          final int[] segCounts = new int[numSegOrds];
           int doc = 0;
           while (doc < maxDoc && (doc = matchingDocs.bits.nextSetBit(doc)) != -1) {
             segValues.setDocument(doc);
             int term = (int) segValues.nextOrd();
             while (term != SortedSetDocValues.NO_MORE_ORDS) {
-              counts[term]++;
+              segCounts[term]++;
               term = (int) segValues.nextOrd();
             }
             ++doc;
           }
-        }
-      }
 
-      @Override
-      public void rollupValues(FacetRequest fr, int ordinal, int[] children, int[] siblings, FacetArrays facetArrays) {
-        // Nothing to do here: we only support flat (dim +
-        // label) facets, and in accumulate we sum up the
-        // count for the dimension.
-      }
+          // Then, migrate to global ords:
+          for(int ord=0;ord<numSegOrds;ord++) {
+            int count = segCounts[ord];
+            if (count != 0) {
+              counts[(int) ordinalMap.getGlobalOrd(segOrd, ord)] += count;
+            }
+          }
+        }
+      } else {
+        // No ord mapping (e.g., single segment index):
+        // just aggregate directly into counts:
 
-      @Override
-      public boolean requiresDocScores() {
-        return false;
+        int doc = 0;
+        while (doc < maxDoc && (doc = matchingDocs.bits.nextSetBit(doc)) != -1) {
+          segValues.setDocument(doc);
+          int term = (int) segValues.nextOrd();
+          while (term != SortedSetDocValues.NO_MORE_ORDS) {
+            counts[term]++;
+            term = (int) segValues.nextOrd();
+          }
+          ++doc;
+        }
       }
-    };
-  }
-
-  /** Keeps highest count results. */
-  static class TopCountPQ extends PriorityQueue<FacetResultNode> {
-    public TopCountPQ(int topN) {
-      super(topN, false);
     }
 
-    @Override
-    protected boolean lessThan(FacetResultNode a, FacetResultNode b) {
-      if (a.value < b.value) {
-        return true;
-      } else if (a.value > b.value) {
-        return false;
-      } else {
-        return a.ordinal > b.ordinal;
-      }
-    }
   }
-
+  
   @Override
   public List<FacetResult> accumulate(List<MatchingDocs> matchingDocs) throws IOException {
 
-    FacetsAggregator aggregator = getAggregator();
-    for (CategoryListParams clp : getCategoryLists()) {
-      for (MatchingDocs md : matchingDocs) {
-        aggregator.aggregate(md, clp, facetArrays);
-      }
+    SortedSetAggregator aggregator = new SortedSetAggregator(field, state, dv);
+    for (MatchingDocs md : matchingDocs) {
+      aggregator.aggregate(md, facetArrays);
     }
 
     // compute top-K
@@ -218,7 +219,7 @@ public class SortedSetDocValuesAccumulat
 
     BytesRef scratch = new BytesRef();
 
-    for(FacetRequest request : searchParams.facetRequests) {
+    for (FacetRequest request : searchParams.facetRequests) {
       String dim = request.categoryPath.components[0];
       SortedSetDocValuesReaderState.OrdRange ordRange = state.getOrdRange(dim);
       // checked in ctor:
@@ -315,4 +316,10 @@ public class SortedSetDocValuesAccumulat
 
     return results;
   }
+  
+  @Override
+  public boolean requiresDocScores() {
+    return false;
+  }
+  
 }

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java Tue Aug 13 04:06:18 2013
@@ -428,7 +428,7 @@ public class DirectoryTaxonomyWriter imp
         Terms terms = ctx.reader().terms(Consts.FULL);
         if (terms != null) {
           termsEnum = terms.iterator(termsEnum);
-          if (termsEnum.seekExact(catTerm, true)) {
+          if (termsEnum.seekExact(catTerm)) {
             // liveDocs=null because the taxonomy has no deletes
             docs = termsEnum.docs(null, docs, 0 /* freqs not required */);
             // if the term was found, we know it has exactly one document.

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/package.html?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/package.html (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/package.html Tue Aug 13 04:06:18 2013
@@ -21,28 +21,33 @@
 <body>
 	<h1>Taxonomy of Categories</h1>
 	
-	Facets are defined using a hierarchy of categories, known as a
-	<i>Taxonomy</i>.
-	
-	<br>
-	For example, in a book store application, a Taxonomy could have the
-	following hierarchy:
-	<p>
+	Facets are defined using a hierarchy of categories, known as a <i>Taxonomy</i>.
+	For example, the taxonomy of a book store application might have the following structure:
 	<ul>
-		<li>Author</li>
-		<ul>
-		<li>Mark Twain</li>
-		<li>J. K. Rowling</li>
-		</ul>
+		<li>Author
+			<ul>
+			<li>Mark Twain</li>
+			<li>J. K. Rowling</li>
+			</ul>
+		</li>
 	</ul>
 	<ul>
-		<li>Date</li>
-		<ul>
-		<li>2010</li>
-		<li>2009</li>
-		</ul>
+		<li>Date
+			<ul>
+			<li>2010
+				<ul>
+				<li>March</li>
+				<li>April</li>
+				</ul>
+			</li>
+			<li>2009</li>
+			</ul>
+		</li>
 	</ul>
 	
-	The <i>Taxonomy</i> translates category-paths into category-ordinal and vice versa.
+	The <i>Taxonomy</i> translates category-paths into interger identifiers (often termed <i>ordinals</i>) and vice versa.
+	The category <code>Author/Mark Twain</code> adds two nodes to the taxonomy: <code>Author</code> and 
+	<code>Author/Mark Twain</code>, each is assigned a different ordinal. The taxonomy maintains the invariant that a 
+	node always has an ordinal that is &lt; all its children.
 </body>
 </html>
\ No newline at end of file

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/util/FacetsPayloadMigrationReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/util/FacetsPayloadMigrationReader.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/util/FacetsPayloadMigrationReader.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/util/FacetsPayloadMigrationReader.java Tue Aug 13 04:06:18 2013
@@ -90,7 +90,7 @@ public class FacetsPayloadMigrationReade
           Terms terms = fields.terms(term.field());
           if (terms != null) {
             TermsEnum te = terms.iterator(null); // no use for reusing
-            if (te.seekExact(term.bytes(), true)) {
+            if (te.seekExact(term.bytes())) {
               // we're not expected to be called for deleted documents
               dpe = te.docsAndPositions(null, null, DocsAndPositionsEnum.FLAG_PAYLOADS);
             }

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/util/TaxonomyMergeUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/util/TaxonomyMergeUtils.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/util/TaxonomyMergeUtils.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/util/TaxonomyMergeUtils.java Tue Aug 13 04:06:18 2013
@@ -45,7 +45,7 @@ public class TaxonomyMergeUtils {
     // merge the taxonomies
     destTaxWriter.addTaxonomy(srcTaxDir, map);
     int ordinalMap[] = map.getMap();
-    DirectoryReader reader = DirectoryReader.open(srcIndexDir, -1);
+    DirectoryReader reader = DirectoryReader.open(srcIndexDir);
     List<AtomicReaderContext> leaves = reader.leaves();
     int numReaders = leaves.size();
     AtomicReader wrappedLeaves[] = new AtomicReader[numReaders];

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/overview.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/overview.html?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/overview.html (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/overview.html Tue Aug 13 04:06:18 2013
@@ -15,12 +15,10 @@
  limitations under the License.
 -->
 <html>
-  <head>
-    <title>
-      facet
-    </title>
-  </head>
-  <body>
-  Provides faceted indexing and search capabilities (checkout the <a href="org/apache/lucene/facet/doc-files/userguide.html">userguide</a>).
-  </body>
+<head><title>facet</title></head>
+<body>
+Provides faceted indexing and search capabilities. Checkout <a href="http://shaierera.blogspot.com/2012/11/lucene-facets-part-1.html">this</a>
+and <a href="http://shaierera.blogspot.com/2012/11/lucene-facets-part-2.html">this</a> blog posts for some overview on the facets module
+as well as source code examples <a href="../demo">here</a>.
+</body>
 </html>

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/associations/AssociationsFacetRequestTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/associations/AssociationsFacetRequestTest.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/associations/AssociationsFacetRequestTest.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/associations/AssociationsFacetRequestTest.java Tue Aug 13 04:06:18 2013
@@ -1,8 +1,6 @@
 package org.apache.lucene.facet.associations;
 
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
@@ -10,9 +8,9 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.facet.FacetTestCase;
 import org.apache.lucene.facet.params.FacetSearchParams;
 import org.apache.lucene.facet.search.FacetResult;
-import org.apache.lucene.facet.search.FacetsAccumulator;
 import org.apache.lucene.facet.search.FacetsAggregator;
 import org.apache.lucene.facet.search.FacetsCollector;
+import org.apache.lucene.facet.search.TaxonomyFacetsAccumulator;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
@@ -69,14 +67,18 @@ public class AssociationsFacetRequestTes
     AssociationsFacetFields assocFacetFields = new AssociationsFacetFields(taxoWriter);
     
     // index documents, 50% have only 'b' and all have 'a'
-    for (int i = 0; i < 100; i++) {
+    for (int i = 0; i < 110; i++) {
       Document doc = new Document();
       CategoryAssociationsContainer associations = new CategoryAssociationsContainer();
-      associations.setAssociation(aint, new CategoryIntAssociation(2));
-      associations.setAssociation(afloat, new CategoryFloatAssociation(0.5f));
-      if (i % 2 == 0) { // 50
-        associations.setAssociation(bint, new CategoryIntAssociation(3));
-        associations.setAssociation(bfloat, new CategoryFloatAssociation(0.2f));
+      // every 11th document is added empty, this used to cause the association
+      // aggregators to go into an infinite loop
+      if (i % 11 != 0) {
+        associations.setAssociation(aint, new CategoryIntAssociation(2));
+        associations.setAssociation(afloat, new CategoryFloatAssociation(0.5f));
+        if (i % 2 == 0) { // 50
+          associations.setAssociation(bint, new CategoryIntAssociation(3));
+          associations.setAssociation(bfloat, new CategoryFloatAssociation(0.2f));
+        }
       }
       assocFacetFields.addFields(doc, associations);
       writer.addDocument(doc);
@@ -103,12 +105,12 @@ public class AssociationsFacetRequestTes
     
     // facet requests for two facets
     FacetSearchParams fsp = new FacetSearchParams(
-        new AssociationIntSumFacetRequest(aint, 10),
-        new AssociationIntSumFacetRequest(bint, 10));
+        new SumIntAssociationFacetRequest(aint, 10),
+        new SumIntAssociationFacetRequest(bint, 10));
     
     Query q = new MatchAllDocsQuery();
     
-    FacetsAccumulator fa = new FacetsAccumulator(fsp, reader, taxo) {
+    TaxonomyFacetsAccumulator fa = new TaxonomyFacetsAccumulator(fsp, reader, taxo) {
       @Override
       public FacetsAggregator getAggregator() {
         return new SumIntAssociationFacetsAggregator();
@@ -135,12 +137,12 @@ public class AssociationsFacetRequestTes
     
     // facet requests for two facets
     FacetSearchParams fsp = new FacetSearchParams(
-        new AssociationFloatSumFacetRequest(afloat, 10),
-        new AssociationFloatSumFacetRequest(bfloat, 10));
+        new SumFloatAssociationFacetRequest(afloat, 10),
+        new SumFloatAssociationFacetRequest(bfloat, 10));
     
     Query q = new MatchAllDocsQuery();
     
-    FacetsAccumulator fa = new FacetsAccumulator(fsp, reader, taxo) {
+    TaxonomyFacetsAccumulator fa = new TaxonomyFacetsAccumulator(fsp, reader, taxo) {
       @Override
       public FacetsAggregator getAggregator() {
         return new SumFloatAssociationFacetsAggregator();
@@ -167,27 +169,14 @@ public class AssociationsFacetRequestTes
     
     // facet requests for two facets
     FacetSearchParams fsp = new FacetSearchParams(
-        new AssociationIntSumFacetRequest(aint, 10),
-        new AssociationIntSumFacetRequest(bint, 10),
-        new AssociationFloatSumFacetRequest(afloat, 10),
-        new AssociationFloatSumFacetRequest(bfloat, 10));
+        new SumIntAssociationFacetRequest(aint, 10),
+        new SumIntAssociationFacetRequest(bint, 10),
+        new SumFloatAssociationFacetRequest(afloat, 10),
+        new SumFloatAssociationFacetRequest(bfloat, 10));
     
     Query q = new MatchAllDocsQuery();
     
-    final SumIntAssociationFacetsAggregator sumInt = new SumIntAssociationFacetsAggregator();
-    final SumFloatAssociationFacetsAggregator sumFloat = new SumFloatAssociationFacetsAggregator();
-    final Map<CategoryPath,FacetsAggregator> aggregators = new HashMap<CategoryPath,FacetsAggregator>();
-    aggregators.put(aint, sumInt);
-    aggregators.put(bint, sumInt);
-    aggregators.put(afloat, sumFloat);
-    aggregators.put(bfloat, sumFloat);
-    FacetsAccumulator fa = new FacetsAccumulator(fsp, reader, taxo) {
-      @Override
-      public FacetsAggregator getAggregator() {
-        return new MultiAssociationsFacetsAggregator(aggregators);
-      }
-    };
-    FacetsCollector fc = FacetsCollector.create(fa);
+    FacetsCollector fc = FacetsCollector.create(fsp, reader, taxo);
     
     IndexSearcher searcher = newSearcher(reader);
     searcher.search(q, fc);
@@ -200,6 +189,6 @@ public class AssociationsFacetRequestTes
     assertEquals("Wrong count for category 'b'!",10f, (float) res.get(3).getFacetResultNode().value, 0.00001);
     
     taxo.close();
-  }  
+  }
   
 }

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/complements/TestFacetsAccumulatorWithComplement.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/complements/TestFacetsAccumulatorWithComplement.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/complements/TestFacetsAccumulatorWithComplement.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/complements/TestFacetsAccumulatorWithComplement.java Tue Aug 13 04:06:18 2013
@@ -4,13 +4,13 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.lucene.facet.FacetTestBase;
+import org.apache.lucene.facet.old.OldFacetsAccumulator;
 import org.apache.lucene.facet.params.FacetIndexingParams;
 import org.apache.lucene.facet.params.FacetSearchParams;
 import org.apache.lucene.facet.search.CountFacetRequest;
 import org.apache.lucene.facet.search.FacetResult;
 import org.apache.lucene.facet.search.FacetResultNode;
 import org.apache.lucene.facet.search.FacetsCollector;
-import org.apache.lucene.facet.search.StandardFacetsAccumulator;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.MultiReader;
@@ -112,8 +112,8 @@ public class TestFacetsAccumulatorWithCo
   /** compute facets with certain facet requests and docs */
   private List<FacetResult> findFacets(boolean withComplement) throws IOException {
     FacetSearchParams fsp = new FacetSearchParams(fip, new CountFacetRequest(new CategoryPath("root","a"), 10));
-    StandardFacetsAccumulator sfa = new StandardFacetsAccumulator(fsp, indexReader, taxoReader);
-    sfa.setComplementThreshold(withComplement ? StandardFacetsAccumulator.FORCE_COMPLEMENT : StandardFacetsAccumulator.DISABLE_COMPLEMENT);
+    OldFacetsAccumulator sfa = new OldFacetsAccumulator(fsp, indexReader, taxoReader);
+    sfa.setComplementThreshold(withComplement ? OldFacetsAccumulator.FORCE_COMPLEMENT : OldFacetsAccumulator.DISABLE_COMPLEMENT);
     FacetsCollector fc = FacetsCollector.create(sfa);
     searcher.search(new MatchAllDocsQuery(), fc);
     

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeAccumulator.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeAccumulator.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeAccumulator.java Tue Aug 13 04:06:18 2013
@@ -17,6 +17,7 @@ package org.apache.lucene.facet.range;
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
@@ -37,13 +38,15 @@ import org.apache.lucene.facet.params.Fa
 import org.apache.lucene.facet.params.FacetSearchParams;
 import org.apache.lucene.facet.search.CountFacetRequest;
 import org.apache.lucene.facet.search.DrillDownQuery;
-import org.apache.lucene.facet.search.DrillSideways.DrillSidewaysResult;
 import org.apache.lucene.facet.search.DrillSideways;
+import org.apache.lucene.facet.search.DrillSideways.DrillSidewaysResult;
 import org.apache.lucene.facet.search.FacetRequest;
 import org.apache.lucene.facet.search.FacetResult;
 import org.apache.lucene.facet.search.FacetResultNode;
 import org.apache.lucene.facet.search.FacetsAccumulator;
 import org.apache.lucene.facet.search.FacetsCollector;
+import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetFields;
+import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
@@ -74,15 +77,12 @@ public class TestRangeAccumulator extend
     IndexReader r = w.getReader();
     w.close();
 
-    FacetSearchParams fsp = new FacetSearchParams(
-                                new RangeFacetRequest<LongRange>("field",
-                                                      new LongRange("less than 10", 0L, true, 10L, false),
-                                                      new LongRange("less than or equal to 10", 0L, true, 10L, true),
-                                                      new LongRange("over 90", 90L, false, 100L, false),
-                                                      new LongRange("90 or above", 90L, true, 100L, false),
-                                                      new LongRange("over 1000", 1000L, false, Long.MAX_VALUE, false)));
-
-    RangeAccumulator a = new RangeAccumulator(fsp, r);
+    RangeAccumulator a = new RangeAccumulator(new RangeFacetRequest<LongRange>("field",
+        new LongRange("less than 10", 0L, true, 10L, false),
+        new LongRange("less than or equal to 10", 0L, true, 10L, true),
+        new LongRange("over 90", 90L, false, 100L, false),
+        new LongRange("90 or above", 90L, true, 100L, false),
+        new LongRange("over 1000", 1000L, false, Long.MAX_VALUE, false)));
     
     FacetsCollector fc = FacetsCollector.create(a);
 
@@ -97,15 +97,15 @@ public class TestRangeAccumulator extend
   }
 
   /** Tests single request that mixes Range and non-Range
-   *  faceting, with DrillSideways. */
-  public void testMixedRangeAndNonRange() throws Exception {
+   *  faceting, with DrillSideways and taxonomy. */
+  public void testMixedRangeAndNonRangeTaxonomy() throws Exception {
     Directory d = newDirectory();
     RandomIndexWriter w = new RandomIndexWriter(random(), d);
     Directory td = newDirectory();
     DirectoryTaxonomyWriter tw = new DirectoryTaxonomyWriter(td, IndexWriterConfig.OpenMode.CREATE);
     FacetFields ff = new FacetFields(tw);
 
-    for(long l=0;l<100;l++) {
+    for (long l = 0; l < 100; l++) {
       Document doc = new Document();
       // For computing range facet counts:
       doc.add(new NumericDocValuesField("field", l));
@@ -122,7 +122,7 @@ public class TestRangeAccumulator extend
       w.addDocument(doc);
     }
 
-    IndexReader r = w.getReader();
+    final IndexReader r = w.getReader();
     w.close();
 
     final TaxonomyReader tr = new DirectoryTaxonomyReader(tw);
@@ -130,32 +130,32 @@ public class TestRangeAccumulator extend
 
     IndexSearcher s = newSearcher(r);
 
-    final FacetSearchParams fsp = new FacetSearchParams(
-                                new CountFacetRequest(new CategoryPath("dim"), 2),
-                                new RangeFacetRequest<LongRange>("field",
-                                                      new LongRange("less than 10", 0L, true, 10L, false),
-                                                      new LongRange("less than or equal to 10", 0L, true, 10L, true),
-                                                      new LongRange("over 90", 90L, false, 100L, false),
-                                                      new LongRange("90 or above", 90L, true, 100L, false),
-                                                      new LongRange("over 1000", 1000L, false, Long.MAX_VALUE, false)));
-
+    final CountFacetRequest countRequest = new CountFacetRequest(new CategoryPath("dim"), 2);
+    final RangeFacetRequest<LongRange> rangeRequest = new RangeFacetRequest<LongRange>("field",
+                          new LongRange("less than 10", 0L, true, 10L, false),
+                          new LongRange("less than or equal to 10", 0L, true, 10L, true),
+                          new LongRange("over 90", 90L, false, 100L, false),
+                          new LongRange("90 or above", 90L, true, 100L, false),
+                          new LongRange("over 1000", 1000L, false, Long.MAX_VALUE, false));
+    FacetSearchParams fsp = new FacetSearchParams(countRequest, rangeRequest);
+    
     final Set<String> dimSeen = new HashSet<String>();
 
     DrillSideways ds = new DrillSideways(s, tr) {
         @Override
         protected FacetsAccumulator getDrillDownAccumulator(FacetSearchParams fsp) {
           checkSeen(fsp);
-          return RangeFacetsAccumulatorWrapper.create(fsp, searcher.getIndexReader(), tr);
+          return FacetsAccumulator.create(fsp, r, tr, null);
         }
 
         @Override
         protected FacetsAccumulator getDrillSidewaysAccumulator(String dim, FacetSearchParams fsp) {
           checkSeen(fsp);
-          return RangeFacetsAccumulatorWrapper.create(fsp, searcher.getIndexReader(), tr);
+          return FacetsAccumulator.create(fsp, r, tr, null);
         }
 
         private void checkSeen(FacetSearchParams fsp) {
-          // Each dim should should up only once, across
+          // Each dim should up only once, across
           // both drillDown and drillSideways requests:
           for(FacetRequest fr : fsp.facetRequests) {
             String dim = fr.categoryPath.components[0];
@@ -204,6 +204,111 @@ public class TestRangeAccumulator extend
     IOUtils.close(tr, td, r, d);
   }
 
+  /** Tests single request that mixes Range and non-Range
+   *  faceting, with DrillSideways and SortedSet. */
+  public void testMixedRangeAndNonRangeSortedSet() throws Exception {
+    assumeTrue("Test requires SortedSetDV support", defaultCodecSupportsSortedSet());
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+    SortedSetDocValuesFacetFields ff = new SortedSetDocValuesFacetFields();
+
+    for (long l = 0; l < 100; l++) {
+      Document doc = new Document();
+      // For computing range facet counts:
+      doc.add(new NumericDocValuesField("field", l));
+      // For drill down by numeric range:
+      doc.add(new LongField("field", l, Field.Store.NO));
+
+      CategoryPath cp;
+      if ((l&3) == 0) {
+        cp = new CategoryPath("dim", "a");
+      } else {
+        cp = new CategoryPath("dim", "b");
+      }
+      ff.addFields(doc, Collections.singletonList(cp));
+      w.addDocument(doc);
+    }
+
+    final IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    final SortedSetDocValuesReaderState state = new SortedSetDocValuesReaderState(s.getIndexReader());
+
+    final CountFacetRequest countRequest = new CountFacetRequest(new CategoryPath("dim"), 2);
+    final RangeFacetRequest<LongRange> rangeRequest = new RangeFacetRequest<LongRange>("field",
+                          new LongRange("less than 10", 0L, true, 10L, false),
+                          new LongRange("less than or equal to 10", 0L, true, 10L, true),
+                          new LongRange("over 90", 90L, false, 100L, false),
+                          new LongRange("90 or above", 90L, true, 100L, false),
+                          new LongRange("over 1000", 1000L, false, Long.MAX_VALUE, false));
+    FacetSearchParams fsp = new FacetSearchParams(countRequest, rangeRequest);
+    
+    final Set<String> dimSeen = new HashSet<String>();
+
+    DrillSideways ds = new DrillSideways(s, state) {
+        @Override
+        protected FacetsAccumulator getDrillDownAccumulator(FacetSearchParams fsp) throws IOException {
+          checkSeen(fsp);
+          return FacetsAccumulator.create(fsp, state, null);
+        }
+
+        @Override
+        protected FacetsAccumulator getDrillSidewaysAccumulator(String dim, FacetSearchParams fsp) throws IOException {
+          checkSeen(fsp);
+          return FacetsAccumulator.create(fsp, state, null);
+        }
+
+        private void checkSeen(FacetSearchParams fsp) {
+          // Each dim should up only once, across
+          // both drillDown and drillSideways requests:
+          for(FacetRequest fr : fsp.facetRequests) {
+            String dim = fr.categoryPath.components[0];
+            assertFalse("dim " + dim + " already seen", dimSeen.contains(dim));
+            dimSeen.add(dim);
+          }
+        }
+
+        @Override
+        protected boolean scoreSubDocsAtOnce() {
+          return random().nextBoolean();
+        }
+      };
+
+    // First search, no drill downs:
+    DrillDownQuery ddq = new DrillDownQuery(FacetIndexingParams.DEFAULT, new MatchAllDocsQuery());
+    DrillSidewaysResult dsr = ds.search(null, ddq, 10, fsp);
+
+    assertEquals(100, dsr.hits.totalHits);
+    assertEquals(2, dsr.facetResults.size());
+    assertEquals("dim (0)\n  b (75)\n  a (25)\n", FacetTestUtils.toSimpleString(dsr.facetResults.get(0)));
+    assertEquals("field (0)\n  less than 10 (10)\n  less than or equal to 10 (11)\n  over 90 (9)\n  90 or above (10)\n  over 1000 (0)\n", FacetTestUtils.toSimpleString(dsr.facetResults.get(1)));
+
+    // Second search, drill down on dim=b:
+    ddq = new DrillDownQuery(FacetIndexingParams.DEFAULT, new MatchAllDocsQuery());
+    ddq.add(new CategoryPath("dim", "b"));
+    dimSeen.clear();
+    dsr = ds.search(null, ddq, 10, fsp);
+
+    assertEquals(75, dsr.hits.totalHits);
+    assertEquals(2, dsr.facetResults.size());
+    assertEquals("dim (0)\n  b (75)\n  a (25)\n", FacetTestUtils.toSimpleString(dsr.facetResults.get(0)));
+    assertEquals("field (0)\n  less than 10 (7)\n  less than or equal to 10 (8)\n  over 90 (7)\n  90 or above (8)\n  over 1000 (0)\n", FacetTestUtils.toSimpleString(dsr.facetResults.get(1)));
+
+    // Third search, drill down on "less than or equal to 10":
+    ddq = new DrillDownQuery(FacetIndexingParams.DEFAULT, new MatchAllDocsQuery());
+    ddq.add("field", NumericRangeQuery.newLongRange("field", 0L, 10L, true, true));
+    dimSeen.clear();
+    dsr = ds.search(null, ddq, 10, fsp);
+
+    assertEquals(11, dsr.hits.totalHits);
+    assertEquals(2, dsr.facetResults.size());
+    assertEquals("dim (0)\n  b (8)\n  a (3)\n", FacetTestUtils.toSimpleString(dsr.facetResults.get(0)));
+    assertEquals("field (0)\n  less than 10 (10)\n  less than or equal to 10 (11)\n  over 90 (9)\n  90 or above (10)\n  over 1000 (0)\n", FacetTestUtils.toSimpleString(dsr.facetResults.get(1)));
+
+    IOUtils.close(r, d);
+  }
+
   public void testBasicDouble() throws Exception {
     Directory d = newDirectory();
     RandomIndexWriter w = new RandomIndexWriter(random(), d);
@@ -211,22 +316,19 @@ public class TestRangeAccumulator extend
     DoubleDocValuesField field = new DoubleDocValuesField("field", 0.0);
     doc.add(field);
     for(long l=0;l<100;l++) {
-      field.setDoubleValue((double) l);
+      field.setDoubleValue(l);
       w.addDocument(doc);
     }
 
     IndexReader r = w.getReader();
     w.close();
 
-    FacetSearchParams fsp = new FacetSearchParams(
-                                new RangeFacetRequest<DoubleRange>("field",
-                                                      new DoubleRange("less than 10", 0.0, true, 10.0, false),
-                                                      new DoubleRange("less than or equal to 10", 0.0, true, 10.0, true),
-                                                      new DoubleRange("over 90", 90.0, false, 100.0, false),
-                                                      new DoubleRange("90 or above", 90.0, true, 100.0, false),
-                                                      new DoubleRange("over 1000", 1000.0, false, Double.POSITIVE_INFINITY, false)));
-
-    RangeAccumulator a = new RangeAccumulator(fsp, r);
+    RangeAccumulator a = new RangeAccumulator(new RangeFacetRequest<DoubleRange>("field",
+        new DoubleRange("less than 10", 0.0, true, 10.0, false),
+        new DoubleRange("less than or equal to 10", 0.0, true, 10.0, true),
+        new DoubleRange("over 90", 90.0, false, 100.0, false),
+        new DoubleRange("90 or above", 90.0, true, 100.0, false),
+        new DoubleRange("over 1000", 1000.0, false, Double.POSITIVE_INFINITY, false)));
     
     FacetsCollector fc = FacetsCollector.create(a);
 
@@ -247,22 +349,19 @@ public class TestRangeAccumulator extend
     FloatDocValuesField field = new FloatDocValuesField("field", 0.0f);
     doc.add(field);
     for(long l=0;l<100;l++) {
-      field.setFloatValue((float) l);
+      field.setFloatValue(l);
       w.addDocument(doc);
     }
 
     IndexReader r = w.getReader();
     w.close();
 
-    FacetSearchParams fsp = new FacetSearchParams(
-                                new RangeFacetRequest<FloatRange>("field",
-                                                      new FloatRange("less than 10", 0.0f, true, 10.0f, false),
-                                                      new FloatRange("less than or equal to 10", 0.0f, true, 10.0f, true),
-                                                      new FloatRange("over 90", 90.0f, false, 100.0f, false),
-                                                      new FloatRange("90 or above", 90.0f, true, 100.0f, false),
-                                                      new FloatRange("over 1000", 1000.0f, false, Float.POSITIVE_INFINITY, false)));
-
-    RangeAccumulator a = new RangeAccumulator(fsp, r);
+    RangeAccumulator a = new RangeAccumulator(new RangeFacetRequest<FloatRange>("field",
+        new FloatRange("less than 10", 0.0f, true, 10.0f, false),
+        new FloatRange("less than or equal to 10", 0.0f, true, 10.0f, true),
+        new FloatRange("over 90", 90.0f, false, 100.0f, false),
+        new FloatRange("90 or above", 90.0f, true, 100.0f, false),
+        new FloatRange("over 1000", 1000.0f, false, Float.POSITIVE_INFINITY, false)));
     
     FacetsCollector fc = FacetsCollector.create(a);
 
@@ -335,8 +434,7 @@ public class TestRangeAccumulator extend
         }
       }
 
-      FacetSearchParams fsp = new FacetSearchParams(new RangeFacetRequest<LongRange>("field", ranges));
-      FacetsCollector fc = FacetsCollector.create(new RangeAccumulator(fsp, r));
+      FacetsCollector fc = FacetsCollector.create(new RangeAccumulator(new RangeFacetRequest<LongRange>("field", ranges)));
       s.search(new MatchAllDocsQuery(), fc);
       List<FacetResult> results = fc.getFacetResults();
       assertEquals(1, results.size());
@@ -350,7 +448,7 @@ public class TestRangeAccumulator extend
         assertEquals("field/r" + rangeID, subNode.label.toString('/'));
         assertEquals(expectedCounts[rangeID], (int) subNode.value);
 
-        LongRange range = (LongRange) ((RangeFacetRequest) results.get(0).getFacetRequest()).ranges[rangeID];
+        LongRange range = (LongRange) ((RangeFacetRequest<?>) results.get(0).getFacetRequest()).ranges[rangeID];
 
         // Test drill-down:
         DrillDownQuery ddq = new DrillDownQuery(FacetIndexingParams.DEFAULT);
@@ -422,8 +520,7 @@ public class TestRangeAccumulator extend
         }
       }
 
-      FacetSearchParams fsp = new FacetSearchParams(new RangeFacetRequest<FloatRange>("field", ranges));
-      FacetsCollector fc = FacetsCollector.create(new RangeAccumulator(fsp, r));
+      FacetsCollector fc = FacetsCollector.create(new RangeAccumulator(new RangeFacetRequest<FloatRange>("field", ranges)));
       s.search(new MatchAllDocsQuery(), fc);
       List<FacetResult> results = fc.getFacetResults();
       assertEquals(1, results.size());
@@ -437,7 +534,7 @@ public class TestRangeAccumulator extend
         assertEquals("field/r" + rangeID, subNode.label.toString('/'));
         assertEquals(expectedCounts[rangeID], (int) subNode.value);
 
-        FloatRange range = (FloatRange) ((RangeFacetRequest) results.get(0).getFacetRequest()).ranges[rangeID];
+        FloatRange range = (FloatRange) ((RangeFacetRequest<?>) results.get(0).getFacetRequest()).ranges[rangeID];
 
         // Test drill-down:
         DrillDownQuery ddq = new DrillDownQuery(FacetIndexingParams.DEFAULT);
@@ -509,8 +606,7 @@ public class TestRangeAccumulator extend
         }
       }
 
-      FacetSearchParams fsp = new FacetSearchParams(new RangeFacetRequest<DoubleRange>("field", ranges));
-      FacetsCollector fc = FacetsCollector.create(new RangeAccumulator(fsp, r));
+      FacetsCollector fc = FacetsCollector.create(new RangeAccumulator(new RangeFacetRequest<DoubleRange>("field", ranges)));
       s.search(new MatchAllDocsQuery(), fc);
       List<FacetResult> results = fc.getFacetResults();
       assertEquals(1, results.size());
@@ -524,7 +620,7 @@ public class TestRangeAccumulator extend
         assertEquals("field/r" + rangeID, subNode.label.toString('/'));
         assertEquals(expectedCounts[rangeID], (int) subNode.value);
 
-        DoubleRange range = (DoubleRange) ((RangeFacetRequest) results.get(0).getFacetRequest()).ranges[rangeID];
+        DoubleRange range = (DoubleRange) ((RangeFacetRequest<?>) results.get(0).getFacetRequest()).ranges[rangeID];
 
         // Test drill-down:
         DrillDownQuery ddq = new DrillDownQuery(FacetIndexingParams.DEFAULT);

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sampling/BaseSampleTestTopK.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sampling/BaseSampleTestTopK.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sampling/BaseSampleTestTopK.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sampling/BaseSampleTestTopK.java Tue Aug 13 04:06:18 2013
@@ -3,18 +3,14 @@ package org.apache.lucene.facet.sampling
 import java.util.List;
 import java.util.Random;
 
+import org.apache.lucene.facet.old.OldFacetsAccumulator;
 import org.apache.lucene.facet.params.FacetIndexingParams;
 import org.apache.lucene.facet.params.FacetSearchParams;
-import org.apache.lucene.facet.sampling.RandomSampler;
-import org.apache.lucene.facet.sampling.RepeatableSampler;
-import org.apache.lucene.facet.sampling.Sampler;
-import org.apache.lucene.facet.sampling.SamplingParams;
 import org.apache.lucene.facet.search.BaseTestTopK;
 import org.apache.lucene.facet.search.FacetRequest;
+import org.apache.lucene.facet.search.FacetRequest.ResultMode;
 import org.apache.lucene.facet.search.FacetResult;
 import org.apache.lucene.facet.search.FacetsCollector;
-import org.apache.lucene.facet.search.StandardFacetsAccumulator;
-import org.apache.lucene.facet.search.FacetRequest.ResultMode;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
@@ -60,7 +56,7 @@ public abstract class BaseSampleTestTopK
     return res;
   }
   
-  protected abstract StandardFacetsAccumulator getSamplingAccumulator(Sampler sampler, TaxonomyReader taxoReader, 
+  protected abstract OldFacetsAccumulator getSamplingAccumulator(Sampler sampler, TaxonomyReader taxoReader, 
       IndexReader indexReader, FacetSearchParams searchParams);
   
   /**
@@ -123,8 +119,8 @@ public abstract class BaseSampleTestTopK
   
   private FacetsCollector samplingCollector(final boolean complement, final Sampler sampler,
       FacetSearchParams samplingSearchParams) {
-    StandardFacetsAccumulator sfa = getSamplingAccumulator(sampler, taxoReader, indexReader, samplingSearchParams);
-    sfa.setComplementThreshold(complement ? StandardFacetsAccumulator.FORCE_COMPLEMENT : StandardFacetsAccumulator.DISABLE_COMPLEMENT);
+    OldFacetsAccumulator sfa = getSamplingAccumulator(sampler, taxoReader, indexReader, samplingSearchParams);
+    sfa.setComplementThreshold(complement ? OldFacetsAccumulator.FORCE_COMPLEMENT : OldFacetsAccumulator.DISABLE_COMPLEMENT);
     return FacetsCollector.create(sfa);
   }
   

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sampling/OversampleWithDepthTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sampling/OversampleWithDepthTest.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sampling/OversampleWithDepthTest.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sampling/OversampleWithDepthTest.java Tue Aug 13 04:06:18 2013
@@ -6,19 +6,15 @@ import java.util.Collections;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.facet.FacetTestCase;
 import org.apache.lucene.facet.index.FacetFields;
+import org.apache.lucene.facet.old.OldFacetsAccumulator;
 import org.apache.lucene.facet.params.FacetIndexingParams;
 import org.apache.lucene.facet.params.FacetSearchParams;
-import org.apache.lucene.facet.sampling.RandomSampler;
-import org.apache.lucene.facet.sampling.Sampler;
-import org.apache.lucene.facet.sampling.SamplingAccumulator;
-import org.apache.lucene.facet.sampling.SamplingParams;
 import org.apache.lucene.facet.search.CountFacetRequest;
 import org.apache.lucene.facet.search.FacetRequest;
+import org.apache.lucene.facet.search.FacetRequest.ResultMode;
 import org.apache.lucene.facet.search.FacetResult;
 import org.apache.lucene.facet.search.FacetResultNode;
 import org.apache.lucene.facet.search.FacetsCollector;
-import org.apache.lucene.facet.search.StandardFacetsAccumulator;
-import org.apache.lucene.facet.search.FacetRequest.ResultMode;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
@@ -116,7 +112,7 @@ public class OversampleWithDepthTest ext
       final SamplingParams params) throws IOException {
     // a FacetsCollector with a sampling accumulator
     Sampler sampler = new RandomSampler(params, random());
-    StandardFacetsAccumulator sfa = new SamplingAccumulator(sampler, fsp, r, tr);
+    OldFacetsAccumulator sfa = new SamplingAccumulator(sampler, fsp, r, tr);
     FacetsCollector fcWithSampling = FacetsCollector.create(sfa);
     
     IndexSearcher s = newSearcher(r);

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sampling/SamplerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sampling/SamplerTest.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sampling/SamplerTest.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sampling/SamplerTest.java Tue Aug 13 04:06:18 2013
@@ -4,12 +4,12 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.facet.FacetTestBase;
+import org.apache.lucene.facet.old.OldFacetsAccumulator;
 import org.apache.lucene.facet.params.FacetIndexingParams;
 import org.apache.lucene.facet.params.FacetSearchParams;
 import org.apache.lucene.facet.search.CountFacetRequest;
 import org.apache.lucene.facet.search.FacetResultNode;
 import org.apache.lucene.facet.search.FacetsCollector;
-import org.apache.lucene.facet.search.StandardFacetsAccumulator;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.junit.After;
@@ -99,7 +99,7 @@ public class SamplerTest extends FacetTe
     
     // Make sure no complements are in action
     accumulator
-        .setComplementThreshold(StandardFacetsAccumulator.DISABLE_COMPLEMENT);
+        .setComplementThreshold(OldFacetsAccumulator.DISABLE_COMPLEMENT);
     
     FacetsCollector fc = FacetsCollector.create(accumulator);
     

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sampling/SamplingAccumulatorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sampling/SamplingAccumulatorTest.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sampling/SamplingAccumulatorTest.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sampling/SamplingAccumulatorTest.java Tue Aug 13 04:06:18 2013
@@ -1,9 +1,7 @@
 package org.apache.lucene.facet.sampling;
 
+import org.apache.lucene.facet.old.OldFacetsAccumulator;
 import org.apache.lucene.facet.params.FacetSearchParams;
-import org.apache.lucene.facet.sampling.Sampler;
-import org.apache.lucene.facet.sampling.SamplingAccumulator;
-import org.apache.lucene.facet.search.StandardFacetsAccumulator;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.util.LuceneTestCase.Slow;
@@ -29,7 +27,7 @@ import org.apache.lucene.util.LuceneTest
 public class SamplingAccumulatorTest extends BaseSampleTestTopK {
 
   @Override
-  protected StandardFacetsAccumulator getSamplingAccumulator(Sampler sampler, TaxonomyReader taxoReader, 
+  protected OldFacetsAccumulator getSamplingAccumulator(Sampler sampler, TaxonomyReader taxoReader, 
       IndexReader indexReader, FacetSearchParams searchParams) {
     return new SamplingAccumulator(sampler, searchParams, indexReader, taxoReader);
   }

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sampling/SamplingWrapperTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sampling/SamplingWrapperTest.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sampling/SamplingWrapperTest.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sampling/SamplingWrapperTest.java Tue Aug 13 04:06:18 2013
@@ -1,13 +1,10 @@
 package org.apache.lucene.facet.sampling;
 
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.util.LuceneTestCase.Slow;
-
+import org.apache.lucene.facet.old.OldFacetsAccumulator;
 import org.apache.lucene.facet.params.FacetSearchParams;
-import org.apache.lucene.facet.sampling.Sampler;
-import org.apache.lucene.facet.sampling.SamplingWrapper;
-import org.apache.lucene.facet.search.StandardFacetsAccumulator;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.util.LuceneTestCase.Slow;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -30,9 +27,9 @@ import org.apache.lucene.facet.taxonomy.
 public class SamplingWrapperTest extends BaseSampleTestTopK {
 
   @Override
-  protected StandardFacetsAccumulator getSamplingAccumulator(Sampler sampler, TaxonomyReader taxoReader, 
+  protected OldFacetsAccumulator getSamplingAccumulator(Sampler sampler, TaxonomyReader taxoReader, 
       IndexReader indexReader, FacetSearchParams searchParams) {
-    return new SamplingWrapper(new StandardFacetsAccumulator(searchParams, indexReader, taxoReader), sampler);
+    return new SamplingWrapper(new OldFacetsAccumulator(searchParams, indexReader, taxoReader), sampler);
   }
   
 }

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/CountingFacetsAggregatorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/CountingFacetsAggregatorTest.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/CountingFacetsAggregatorTest.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/CountingFacetsAggregatorTest.java Tue Aug 13 04:06:18 2013
@@ -269,7 +269,7 @@ public class CountingFacetsAggregatorTes
     IOUtils.close(indexWriter, taxoWriter);
   }
   
-  private FacetsAccumulator randomAccumulator(FacetSearchParams fsp, IndexReader indexReader, TaxonomyReader taxoReader) {
+  private TaxonomyFacetsAccumulator randomAccumulator(FacetSearchParams fsp, IndexReader indexReader, TaxonomyReader taxoReader) {
     final FacetsAggregator aggregator;
     double val = random().nextDouble();
     if (val < 0.6) {
@@ -279,7 +279,7 @@ public class CountingFacetsAggregatorTes
     } else {
       aggregator = new CachedOrdsCountingFacetsAggregator();
     }
-    return new FacetsAccumulator(fsp, indexReader, taxoReader) {
+    return new TaxonomyFacetsAccumulator(fsp, indexReader, taxoReader) {
       @Override
       public FacetsAggregator getAggregator() {
         return aggregator;

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/FacetResultTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/FacetResultTest.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/FacetResultTest.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/FacetResultTest.java Tue Aug 13 04:06:18 2013
@@ -116,7 +116,7 @@ public class FacetResultTest extends Fac
         @Override
         protected FacetsAccumulator getDrillSidewaysAccumulator(String dim, FacetSearchParams fsp) throws IOException {
           FacetsAccumulator fa = super.getDrillSidewaysAccumulator(dim, fsp);
-          dimArrays.put(dim, fa.facetArrays);
+          dimArrays.put(dim, ((TaxonomyFacetsAccumulator) fa).facetArrays);
           return fa;
         }
       };

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java Tue Aug 13 04:06:18 2013
@@ -41,7 +41,6 @@ import org.apache.lucene.facet.index.Fac
 import org.apache.lucene.facet.params.FacetIndexingParams;
 import org.apache.lucene.facet.params.FacetSearchParams;
 import org.apache.lucene.facet.search.DrillSideways.DrillSidewaysResult;
-import org.apache.lucene.facet.sortedset.SortedSetDocValuesAccumulator;
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetFields;
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
@@ -62,8 +61,8 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.SortField.Type;
 import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.SortField.Type;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
@@ -336,6 +335,8 @@ public class TestDrillSideways extends F
     String id;
     String contentToken;
 
+    public Doc() {}
+    
     // -1 if the doc is missing this dim, else the index
     // -into the values for this dim:
     int[] dims;
@@ -790,17 +791,7 @@ public class TestDrillSideways extends F
       Sort sort = new Sort(new SortField("id", SortField.Type.STRING));
       DrillSideways ds;
       if (doUseDV) {
-        ds = new DrillSideways(s, null) {
-            @Override
-            protected FacetsAccumulator getDrillDownAccumulator(FacetSearchParams fsp) throws IOException {
-              return new SortedSetDocValuesAccumulator(fsp, sortedSetDVState);
-            }
-
-            @Override
-            protected FacetsAccumulator getDrillSidewaysAccumulator(String dim, FacetSearchParams fsp) throws IOException {
-              return new SortedSetDocValuesAccumulator(fsp, sortedSetDVState);
-            }
-          };
+        ds = new DrillSideways(s, sortedSetDVState);
       } else {
         ds = new DrillSideways(s, tr);
       }
@@ -881,6 +872,7 @@ public class TestDrillSideways extends F
     List<Doc> hits;
     int[][] counts;
     int[] uniqueCounts;
+    public SimpleFacetResult() {}
   }
   
   private int[] getTopNOrds(final int[] counts, final String[] values, int topN) {

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/TestFacetsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/TestFacetsCollector.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/TestFacetsCollector.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/TestFacetsCollector.java Tue Aug 13 04:06:18 2013
@@ -3,9 +3,7 @@ package org.apache.lucene.facet.search;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
@@ -13,6 +11,8 @@ import org.apache.lucene.document.Field.
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.facet.FacetTestCase;
 import org.apache.lucene.facet.index.FacetFields;
+import org.apache.lucene.facet.old.AdaptiveFacetsAccumulator;
+import org.apache.lucene.facet.old.OldFacetsAccumulator;
 import org.apache.lucene.facet.params.CategoryListParams;
 import org.apache.lucene.facet.params.FacetIndexingParams;
 import org.apache.lucene.facet.params.FacetSearchParams;
@@ -90,7 +90,7 @@ public class TestFacetsCollector extends
     DirectoryTaxonomyReader taxo = new DirectoryTaxonomyReader(taxoDir);
     
     FacetSearchParams sParams = new FacetSearchParams(new SumScoreFacetRequest(new CategoryPath("a"), 10));
-    FacetsAccumulator fa = new FacetsAccumulator(sParams, r, taxo) {
+    TaxonomyFacetsAccumulator fa = new TaxonomyFacetsAccumulator(sParams, r, taxo) {
       @Override
       public FacetsAggregator getAggregator() {
         return new SumScoreFacetsAggregator();
@@ -181,18 +181,7 @@ public class TestFacetsCollector extends
         new CountFacetRequest(new CategoryPath("a"), 10), 
         new SumScoreFacetRequest(new CategoryPath("b"), 10));
     
-    Map<CategoryListParams,FacetsAggregator> aggregators = new HashMap<CategoryListParams,FacetsAggregator>();
-    aggregators.put(fip.getCategoryListParams(new CategoryPath("a")), new FastCountingFacetsAggregator());
-    aggregators.put(fip.getCategoryListParams(new CategoryPath("b")), new SumScoreFacetsAggregator());
-    final FacetsAggregator aggregator = new PerCategoryListAggregator(aggregators, fip);
-    FacetsAccumulator fa = new FacetsAccumulator(sParams, r, taxo) {
-      @Override
-      public FacetsAggregator getAggregator() {
-        return aggregator;
-      }
-    };
-    
-    FacetsCollector fc = FacetsCollector.create(fa);
+    FacetsCollector fc = FacetsCollector.create(sParams, r, taxo);
     TopScoreDocCollector topDocs = TopScoreDocCollector.create(10, false);
     newSearcher(r).search(new MatchAllDocsQuery(), MultiCollector.wrap(fc, topDocs));
     
@@ -231,7 +220,7 @@ public class TestFacetsCollector extends
     
     FacetSearchParams fsp = new FacetSearchParams(new CountFacetRequest(CategoryPath.EMPTY, 10));
     
-    final FacetsAccumulator fa = random().nextBoolean() ? new FacetsAccumulator(fsp, r, taxo) : new StandardFacetsAccumulator(fsp, r, taxo);
+    final TaxonomyFacetsAccumulator fa = random().nextBoolean() ? new TaxonomyFacetsAccumulator(fsp, r, taxo) : new OldFacetsAccumulator(fsp, r, taxo);
     FacetsCollector fc = FacetsCollector.create(fa);
     newSearcher(r).search(new MatchAllDocsQuery(), fc);
     
@@ -265,7 +254,7 @@ public class TestFacetsCollector extends
     FacetSearchParams fsp = new FacetSearchParams(
         new CountFacetRequest(new CategoryPath("a"), 10), 
         new CountFacetRequest(new CategoryPath("b"), 10));
-    final FacetsAccumulator fa = random().nextBoolean() ? new FacetsAccumulator(fsp, r, taxo) : new StandardFacetsAccumulator(fsp, r, taxo);
+    final TaxonomyFacetsAccumulator fa = random().nextBoolean() ? new TaxonomyFacetsAccumulator(fsp, r, taxo) : new OldFacetsAccumulator(fsp, r, taxo);
     final FacetsCollector fc = FacetsCollector.create(fa);
     newSearcher(r).search(new MatchAllDocsQuery(), fc);
     
@@ -297,7 +286,7 @@ public class TestFacetsCollector extends
     FacetSearchParams fsp = new FacetSearchParams(
         new CountFacetRequest(new CategoryPath("a"), 10), 
         new CountFacetRequest(new CategoryPath("b"), 10));
-    final FacetsAccumulator fa = random().nextBoolean() ? new FacetsAccumulator(fsp, r, taxo) : new StandardFacetsAccumulator(fsp, r, taxo);
+    final TaxonomyFacetsAccumulator fa = random().nextBoolean() ? new TaxonomyFacetsAccumulator(fsp, r, taxo) : new OldFacetsAccumulator(fsp, r, taxo);
     final FacetsCollector fc = FacetsCollector.create(fa);
     // this should populate the cached results, but doing search should clear the cache
     fc.getFacetResults();
@@ -338,7 +327,7 @@ public class TestFacetsCollector extends
 
     // assert IntFacetResultHandler
     FacetSearchParams fsp = new FacetSearchParams(new CountFacetRequest(new CategoryPath("a"), 10));
-    FacetsAccumulator fa = random().nextBoolean() ? new FacetsAccumulator(fsp, r, taxo) : new StandardFacetsAccumulator(fsp, r, taxo);
+    TaxonomyFacetsAccumulator fa = random().nextBoolean() ? new TaxonomyFacetsAccumulator(fsp, r, taxo) : new OldFacetsAccumulator(fsp, r, taxo);
     FacetsCollector fc = FacetsCollector.create(fa);
     newSearcher(r).search(new MatchAllDocsQuery(), fc);
     assertTrue("invalid ordinal for child node: 0", 0 != fc.getFacetResults().get(0).getFacetResultNode().subResults.get(0).ordinal);
@@ -346,14 +335,14 @@ public class TestFacetsCollector extends
     // assert IntFacetResultHandler
     fsp = new FacetSearchParams(new SumScoreFacetRequest(new CategoryPath("a"), 10));
     if (random().nextBoolean()) {
-      fa = new FacetsAccumulator(fsp, r, taxo) {
+      fa = new TaxonomyFacetsAccumulator(fsp, r, taxo) {
         @Override
         public FacetsAggregator getAggregator() {
           return new SumScoreFacetsAggregator();
         }
       };
     } else {
-      fa = new StandardFacetsAccumulator(fsp, r, taxo);
+      fa = new OldFacetsAccumulator(fsp, r, taxo);
     }
     fc = FacetsCollector.create(fa);
     newSearcher(r).search(new MatchAllDocsQuery(), fc);
@@ -387,7 +376,7 @@ public class TestFacetsCollector extends
     CountFacetRequest cfr = new CountFacetRequest(new CategoryPath("a"), 2);
     cfr.setResultMode(random().nextBoolean() ? ResultMode.GLOBAL_FLAT : ResultMode.PER_NODE_IN_TREE);
     FacetSearchParams fsp = new FacetSearchParams(cfr);
-    final FacetsAccumulator fa = random().nextBoolean() ? new FacetsAccumulator(fsp, r, taxo) : new StandardFacetsAccumulator(fsp, r, taxo);
+    final TaxonomyFacetsAccumulator fa = random().nextBoolean() ? new TaxonomyFacetsAccumulator(fsp, r, taxo) : new OldFacetsAccumulator(fsp, r, taxo);
     FacetsCollector fc = FacetsCollector.create(fa);
     newSearcher(r).search(new MatchAllDocsQuery(), fc);
     
@@ -426,15 +415,15 @@ public class TestFacetsCollector extends
     }
     final Sampler sampler = new RandomSampler(sampleParams, random());
     
-    FacetsAccumulator[] accumulators = new FacetsAccumulator[] {
-      new FacetsAccumulator(fsp, indexReader, taxoReader),
-      new StandardFacetsAccumulator(fsp, indexReader, taxoReader),
+    TaxonomyFacetsAccumulator[] accumulators = new TaxonomyFacetsAccumulator[] {
+      new TaxonomyFacetsAccumulator(fsp, indexReader, taxoReader),
+      new OldFacetsAccumulator(fsp, indexReader, taxoReader),
       new SamplingAccumulator(sampler, fsp, indexReader, taxoReader),
       new AdaptiveFacetsAccumulator(fsp, indexReader, taxoReader),
-      new SamplingWrapper(new StandardFacetsAccumulator(fsp, indexReader, taxoReader), sampler)
+      new SamplingWrapper(new OldFacetsAccumulator(fsp, indexReader, taxoReader), sampler)
     };
     
-    for (FacetsAccumulator fa : accumulators) {
+    for (TaxonomyFacetsAccumulator fa : accumulators) {
       FacetsCollector fc = FacetsCollector.create(fa);
       searcher.search(new MatchAllDocsQuery(), fc);
       List<FacetResult> facetResults = fc.getFacetResults();
@@ -444,20 +433,19 @@ public class TestFacetsCollector extends
     
     try {
       // SortedSetDocValuesAccumulator cannot even be created in such state
-      assertNull(new SortedSetDocValuesAccumulator(fsp, new SortedSetDocValuesReaderState(indexReader)));
+      assertNull(new SortedSetDocValuesAccumulator(new SortedSetDocValuesReaderState(indexReader), fsp));
       // if this ever changes, make sure FacetResultNode is labeled correctly 
       fail("should not have succeeded to execute a request over a category which wasn't indexed as SortedSetDVField");
     } catch (IllegalArgumentException e) {
       // expected
     }
 
-    fsp = new FacetSearchParams(new RangeFacetRequest<LongRange>("f", new LongRange("grr", 0, true, 1, true)));
-    RangeAccumulator ra = new RangeAccumulator(fsp, indexReader);
+    RangeAccumulator ra = new RangeAccumulator(new RangeFacetRequest<LongRange>("f", new LongRange("grr", 0, true, 1, true)));
     FacetsCollector fc = FacetsCollector.create(ra);
     searcher.search(new MatchAllDocsQuery(), fc);
     List<FacetResult> facetResults = fc.getFacetResults();
     assertNotNull(facetResults);
-    assertEquals("incorrect label returned for RangeAccumulator", fsp.facetRequests.get(0).categoryPath, facetResults.get(0).getFacetResultNode().label);
+    assertEquals("incorrect label returned for RangeAccumulator", new CategoryPath("f"), facetResults.get(0).getFacetResultNode().label);
 
     IOUtils.close(indexReader, taxoReader);
 

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKInEachNodeResultHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKInEachNodeResultHandler.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKInEachNodeResultHandler.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKInEachNodeResultHandler.java Tue Aug 13 04:06:18 2013
@@ -11,6 +11,7 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.facet.FacetTestCase;
 import org.apache.lucene.facet.index.FacetFields;
+import org.apache.lucene.facet.old.OldFacetsAccumulator;
 import org.apache.lucene.facet.params.FacetIndexingParams;
 import org.apache.lucene.facet.params.FacetSearchParams;
 import org.apache.lucene.facet.search.FacetRequest.ResultMode;
@@ -150,8 +151,8 @@ public class TestTopKInEachNodeResultHan
       FacetSearchParams facetSearchParams = new FacetSearchParams(iParams, facetRequests);
       
       FacetArrays facetArrays = new FacetArrays(PartitionsUtils.partitionSize(facetSearchParams.indexingParams, tr));
-      StandardFacetsAccumulator sfa = new StandardFacetsAccumulator(facetSearchParams, is.getIndexReader(), tr, facetArrays);
-      sfa.setComplementThreshold(StandardFacetsAccumulator.DISABLE_COMPLEMENT);
+      OldFacetsAccumulator sfa = new OldFacetsAccumulator(facetSearchParams, is.getIndexReader(), tr, facetArrays);
+      sfa.setComplementThreshold(OldFacetsAccumulator.DISABLE_COMPLEMENT);
       FacetsCollector fc = FacetsCollector.create(sfa);
       
       is.search(q, fc);
@@ -183,7 +184,7 @@ public class TestTopKInEachNodeResultHan
       }
       // now rearrange
       double [] expectedValues00 = { 6.0, 1.0, 5.0, 3.0, 2.0 };
-      fr = sfa.createFacetResultsHandler(cfra23).rearrangeFacetResult(fr);
+      fr = sfa.createFacetResultsHandler(cfra23, sfa.createOrdinalValueResolver(cfra23)).rearrangeFacetResult(fr);
       i = 0;
       for (FacetResultNode node : parentRes.subResults) {
         assertEquals(expectedValues00[i++], node.value, Double.MIN_VALUE);

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKResultsHandlerRandom.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKResultsHandlerRandom.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKResultsHandlerRandom.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKResultsHandlerRandom.java Tue Aug 13 04:06:18 2013
@@ -4,6 +4,7 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
 
+import org.apache.lucene.facet.old.OldFacetsAccumulator;
 import org.apache.lucene.facet.params.FacetIndexingParams;
 import org.apache.lucene.facet.params.FacetSearchParams;
 import org.apache.lucene.search.MatchAllDocsQuery;
@@ -33,8 +34,8 @@ public class TestTopKResultsHandlerRando
       throws IOException {
     Query q = new MatchAllDocsQuery();
     FacetSearchParams facetSearchParams = searchParamsWithRequests(numResults, fip);
-    StandardFacetsAccumulator sfa = new StandardFacetsAccumulator(facetSearchParams, indexReader, taxoReader);
-    sfa.setComplementThreshold(doComplement ? StandardFacetsAccumulator.FORCE_COMPLEMENT : StandardFacetsAccumulator.DISABLE_COMPLEMENT);
+    OldFacetsAccumulator sfa = new OldFacetsAccumulator(facetSearchParams, indexReader, taxoReader);
+    sfa.setComplementThreshold(doComplement ? OldFacetsAccumulator.FORCE_COMPLEMENT : OldFacetsAccumulator.DISABLE_COMPLEMENT);
     FacetsCollector fc = FacetsCollector.create(sfa);
     searcher.search(q, fc);
     List<FacetResult> facetResults = fc.getFacetResults();

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sortedset/TestSortedSetDocValuesFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sortedset/TestSortedSetDocValuesFacets.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sortedset/TestSortedSetDocValuesFacets.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/sortedset/TestSortedSetDocValuesFacets.java Tue Aug 13 04:06:18 2013
@@ -112,7 +112,7 @@ public class TestSortedSetDocValuesFacet
     //SortedSetDocValuesCollector c = new SortedSetDocValuesCollector(state);
     //SortedSetDocValuesCollectorMergeBySeg c = new SortedSetDocValuesCollectorMergeBySeg(state);
 
-    FacetsCollector c = FacetsCollector.create(new SortedSetDocValuesAccumulator(fsp, state));
+    FacetsCollector c = FacetsCollector.create(new SortedSetDocValuesAccumulator(state, fsp));
 
     searcher.search(new MatchAllDocsQuery(), c);
 
@@ -177,7 +177,7 @@ public class TestSortedSetDocValuesFacet
 
     FacetSearchParams fsp = new FacetSearchParams(requests);
     
-    FacetsCollector c = FacetsCollector.create(new SortedSetDocValuesAccumulator(fsp, state));
+    FacetsCollector c = FacetsCollector.create(new SortedSetDocValuesAccumulator(state, fsp));
 
     searcher.search(new MatchAllDocsQuery(), c);
 

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/cl2o/TestCharBlockArray.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/cl2o/TestCharBlockArray.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/cl2o/TestCharBlockArray.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/cl2o/TestCharBlockArray.java Tue Aug 13 04:06:18 2013
@@ -11,6 +11,7 @@ import java.nio.charset.CodingErrorActio
 
 import org.apache.lucene.facet.FacetTestCase;
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util._TestUtil;
 import org.junit.Test;
 
 /*
@@ -83,7 +84,8 @@ public class TestCharBlockArray extends 
 
     assertEqualsInternal("GrowingCharArray<->StringBuilder mismatch.", builder, array);
 
-    File f = new File("GrowingCharArrayTest.tmp");
+    File tempDir = _TestUtil.getTempDir("growingchararray");
+    File f = new File(tempDir, "GrowingCharArrayTest.tmp");
     BufferedOutputStream out = new BufferedOutputStream(new FileOutputStream(f));
     array.flush(out);
     out.flush();

Modified: lucene/dev/branches/lucene3069/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java (original)
+++ lucene/dev/branches/lucene3069/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java Tue Aug 13 04:06:18 2013
@@ -303,7 +303,7 @@ public abstract class TermGroupFacetColl
 
         int facetOrd;
         if (groupedFacetHit.facetValue != null) {
-          if (facetOrdTermsEnum == null || !facetOrdTermsEnum.seekExact(groupedFacetHit.facetValue, true)) {
+          if (facetOrdTermsEnum == null || !facetOrdTermsEnum.seekExact(groupedFacetHit.facetValue)) {
             continue;
           }
           facetOrd = (int) facetOrdTermsEnum.ord();
@@ -319,7 +319,7 @@ public abstract class TermGroupFacetColl
       if (facetPrefix != null) {
         TermsEnum.SeekStatus seekStatus;
         if (facetOrdTermsEnum != null) {
-          seekStatus = facetOrdTermsEnum.seekCeil(facetPrefix, true);
+          seekStatus = facetOrdTermsEnum.seekCeil(facetPrefix);
         } else {
           seekStatus = TermsEnum.SeekStatus.END;
         }
@@ -334,7 +334,7 @@ public abstract class TermGroupFacetColl
 
         BytesRef facetEndPrefix = BytesRef.deepCopyOf(facetPrefix);
         facetEndPrefix.append(UnicodeUtil.BIG_TERM);
-        seekStatus = facetOrdTermsEnum.seekCeil(facetEndPrefix, true);
+        seekStatus = facetOrdTermsEnum.seekCeil(facetEndPrefix);
         if (seekStatus != TermsEnum.SeekStatus.END) {
           endFacetOrd = (int) facetOrdTermsEnum.ord();
         } else {

Modified: lucene/dev/branches/lucene3069/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java (original)
+++ lucene/dev/branches/lucene3069/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java Tue Aug 13 04:06:18 2013
@@ -282,7 +282,7 @@ public class WeightedSpanTermExtractor {
       TreeSet<Term> extractedTerms = new TreeSet<Term>();
       q.extractTerms(extractedTerms);
       for (Term term : extractedTerms) {
-        termContexts.put(term, TermContext.build(context, term, true));
+        termContexts.put(term, TermContext.build(context, term));
       }
       Bits acceptDocs = context.reader().getLiveDocs();
       final Spans spans = q.getSpans(context, acceptDocs, termContexts);

Modified: lucene/dev/branches/lucene3069/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java (original)
+++ lucene/dev/branches/lucene3069/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java Tue Aug 13 04:06:18 2013
@@ -459,7 +459,7 @@ public class PostingsHighlighter {
         continue;
       } else if (de == null) {
         postings[i] = EMPTY; // initially
-        if (!termsEnum.seekExact(terms[i], true)) {
+        if (!termsEnum.seekExact(terms[i])) {
           continue; // term not found
         }
         de = postings[i] = termsEnum.docsAndPositions(null, null, DocsAndPositionsEnum.FLAG_OFFSETS);
@@ -506,6 +506,13 @@ public class PostingsHighlighter {
         throw new IllegalArgumentException("field '" + field + "' was indexed without offsets, cannot highlight");
       }
       int end = dp.endOffset();
+      // LUCENE-5166: this hit would span the content limit... however more valid 
+      // hits may exist (they are sorted by start). so we pretend like we never 
+      // saw this term, it won't cause a passage to be added to passageQueue or anything.
+      assert EMPTY.startOffset() == Integer.MAX_VALUE;
+      if (start < contentLength && end > contentLength) {
+        continue;
+      }
       if (start >= current.endOffset) {
         if (current.startOffset >= 0) {
           // finalize current

Modified: lucene/dev/branches/lucene3069/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldPhraseList.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldPhraseList.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldPhraseList.java (original)
+++ lucene/dev/branches/lucene3069/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldPhraseList.java Tue Aug 13 04:06:18 2013
@@ -60,50 +60,47 @@ public class FieldPhraseList {
   public FieldPhraseList( FieldTermStack fieldTermStack, FieldQuery fieldQuery, int phraseLimit ){
     final String field = fieldTermStack.getFieldName();
 
-    LinkedList<TermInfo> phraseCandidate = new LinkedList<TermInfo>();
-    QueryPhraseMap currMap = null;
-    QueryPhraseMap nextMap = null;
-    while( !fieldTermStack.isEmpty() && (phraseList.size() < phraseLimit) )
-    {      
-      phraseCandidate.clear();
-
-      TermInfo ti = fieldTermStack.pop();
-      currMap = fieldQuery.getFieldTermMap( field, ti.getText() );
+    QueryPhraseMap qpm = fieldQuery.getRootMap(field);
+    if (qpm != null) {
+      LinkedList<TermInfo> phraseCandidate = new LinkedList<TermInfo>();
+      extractPhrases(fieldTermStack.termList, qpm, phraseCandidate, 0);
+      assert phraseCandidate.size() == 0;
+    }
+  }
 
-      // if not found, discard top TermInfo from stack, then try next element
-      if( currMap == null ) continue;
-      
-      // if found, search the longest phrase
-      phraseCandidate.add( ti );
-      while( true ){
-        ti = fieldTermStack.pop();
-        nextMap = null;
-        if( ti != null )
-          nextMap = currMap.getTermMap( ti.getText() );
-        if( ti == null || nextMap == null ){
-          if( ti != null ) 
-            fieldTermStack.push( ti );
-          if( currMap.isValidTermOrPhrase( phraseCandidate ) ){
-            addIfNoOverlap( new WeightedPhraseInfo( phraseCandidate, currMap.getBoost(), currMap.getTermOrPhraseNumber() ) );
-          }
-          else{
-            while( phraseCandidate.size() > 1 ){
-              fieldTermStack.push( phraseCandidate.removeLast() );
-              currMap = fieldQuery.searchPhrase( field, phraseCandidate );
-              if( currMap != null ){
-                addIfNoOverlap( new WeightedPhraseInfo( phraseCandidate, currMap.getBoost(), currMap.getTermOrPhraseNumber() ) );
-                break;
-              }
-            }
-          }
-          break;
-        }
-        else{
-          phraseCandidate.add( ti );
-          currMap = nextMap;
+  void extractPhrases(LinkedList<TermInfo> terms, QueryPhraseMap currMap, LinkedList<TermInfo> phraseCandidate, int longest) {
+    if (terms.isEmpty()) {
+      if (longest > 0) {
+        addIfNoOverlap( new WeightedPhraseInfo( phraseCandidate.subList(0, longest), currMap.getBoost(), currMap.getTermOrPhraseNumber() ) );
+      }
+      return;
+    }
+    ArrayList<TermInfo> samePositionTerms = new ArrayList<TermInfo>();
+    do {
+      samePositionTerms.add(terms.pop());
+    } while (!terms.isEmpty() && terms.get(0).getPosition() == samePositionTerms.get(0).getPosition());
+
+    // try all next terms at the same position
+    for (TermInfo nextTerm : samePositionTerms) {
+      QueryPhraseMap nextMap = currMap.getTermMap(nextTerm.getText());
+      if (nextMap != null) {
+        phraseCandidate.add(nextTerm);
+        int l = longest;
+        if(nextMap.isValidTermOrPhrase( phraseCandidate ) ){
+          l = phraseCandidate.size();
         }
+        extractPhrases(terms, nextMap, phraseCandidate, l);
+        phraseCandidate.removeLast();
       }
     }
+
+    // ignore the next term
+    extractPhrases(terms, currMap, phraseCandidate, longest);
+
+    // add terms back
+    for (TermInfo nextTerm : samePositionTerms) {
+      terms.push(nextTerm);
+    }
   }
 
   public void addIfNoOverlap( WeightedPhraseInfo wpi ){
@@ -159,11 +156,11 @@ public class FieldPhraseList {
       return termsInfos;
     }
 
-    public WeightedPhraseInfo( LinkedList<TermInfo> terms, float boost ){
+    public WeightedPhraseInfo( List<TermInfo> terms, float boost ){
       this( terms, boost, 0 );
     }
     
-    public WeightedPhraseInfo( LinkedList<TermInfo> terms, float boost, int seqnum ){
+    public WeightedPhraseInfo( List<TermInfo> terms, float boost, int seqnum ){
       this.boost = boost;
       this.seqnum = seqnum;