lucene-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mikemcc...@apache.org
Subject svn commit: r1542713 - in /lucene/dev/branches/lucene5339: ./ lucene/facet/src/java/org/apache/lucene/facet/search/ lucene/facet/src/java/org/apache/lucene/facet/simple/ lucene/facet/src/test/org/apache/lucene/facet/simple/
Date Sun, 17 Nov 2013 12:28:18 GMT
Author: mikemccand
Date: Sun Nov 17 12:28:18 2013
New Revision: 1542713

URL: http://svn.apache.org/r1542713
Log:
LUCENE-5339: cutover DrillSideways

Added:
    lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/MultiFacets.java   (with props)
    lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillSideways.java
      - copied, changed from r1542024, lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java
    lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillSidewaysCollector.java
      - copied, changed from r1542024, lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysCollector.java
    lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillSidewaysQuery.java
      - copied, changed from r1542024, lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysQuery.java
    lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillSidewaysScorer.java
      - copied, changed from r1542024, lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysScorer.java
    lucene/dev/branches/lucene5339/lucene/facet/src/test/org/apache/lucene/facet/simple/TestSimpleDrillSideways.java
      - copied, changed from r1542066, lucene/dev/branches/lucene5339/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java
Modified:
    lucene/dev/branches/lucene5339/TODO
    lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java
    lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillDownQuery.java
    lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleFacetResult.java
    lucene/dev/branches/lucene5339/lucene/facet/src/test/org/apache/lucene/facet/simple/TestTaxonomyFacets.java

Modified: lucene/dev/branches/lucene5339/TODO
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/TODO?rev=1542713&r1=1542712&r2=1542713&view=diff
==============================================================================
--- lucene/dev/branches/lucene5339/TODO (original)
+++ lucene/dev/branches/lucene5339/TODO Sun Nov 17 12:28:18 2013
@@ -2,6 +2,7 @@ nocommit this!
 
 TODO
   - associations
+  - simplify ddq api
   - SSDVValueSourceFacets?
   - we could put more stuff into the "schema", e.g. this field is
     sorted-set-DV and that one is taxo?

Modified: lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java?rev=1542713&r1=1542712&r2=1542713&view=diff
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java (original)
+++ lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java Sun Nov 17 12:28:18 2013
@@ -180,7 +180,7 @@ public class DrillSideways {
       // Just do ordinary search when there are no drill-downs:
       FacetsCollector c = FacetsCollector.create(getDrillDownAccumulator(fsp));
       searcher.search(query, MultiCollector.wrap(hitCollector, c));
-      return new DrillSidewaysResult(c.getFacetResults(), null);      
+      return new DrillSidewaysResult(c.getFacetResults(), null);
     }
 
     List<FacetRequest> ddRequests = new ArrayList<FacetRequest>();

Added: lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/MultiFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/MultiFacets.java?rev=1542713&view=auto
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/MultiFacets.java (added)
+++ lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/MultiFacets.java Sun Nov 17 12:28:18 2013
@@ -0,0 +1,56 @@
+package org.apache.lucene.facet.simple;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+/** Maps specified dims to provided Facets impls; else, uses
+ *  the default Facets impl. */
+public class MultiFacets extends Facets {
+  private final Map<String,Facets> dimToFacets;
+  private final Facets defaultFacets;
+
+  public MultiFacets(Map<String,Facets> dimToFacets, Facets defaultFacets) {
+    this.dimToFacets = dimToFacets;
+    this.defaultFacets = defaultFacets;
+  }
+
+  public SimpleFacetResult getTopChildren(int topN, String dim, String... path) throws IOException {
+    Facets facets = dimToFacets.get(dim);
+    if (facets == null) {
+      facets = defaultFacets;
+    }
+    return facets.getTopChildren(topN, dim, path);
+  }
+
+  public Number getSpecificValue(String dim, String... path) throws IOException {
+    Facets facets = dimToFacets.get(dim);
+    if (facets == null) {
+      facets = defaultFacets;
+    }
+    return facets.getSpecificValue(dim, path);
+  }
+
+  public List<SimpleFacetResult> getAllDims(int topN) throws IOException {
+    // nocommit can/should we impl this?  ie, sparse
+    // faceting after drill sideways
+    throw new UnsupportedOperationException();
+  }
+}

Modified: lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillDownQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillDownQuery.java?rev=1542713&r1=1542712&r2=1542713&view=diff
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillDownQuery.java (original)
+++ lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillDownQuery.java Sun Nov 17 12:28:18 2013
@@ -118,17 +118,17 @@ public final class SimpleDrillDownQuery 
    * Adds one dimension of drill downs; if you pass multiple values they are
    * OR'd, and then the entire dimension is AND'd against the base query.
    */
-  // nocommit can we remove CatPath here?
+  // nocommit can we remove FacetLabel here?
   public void add(FacetLabel... paths) {
     add(FacetsConfig.DEFAULT_INDEXED_FIELD_NAME, Constants.DEFAULT_DELIM_CHAR, paths);
   }
 
-  // nocommit can we remove CatPath here?
+  // nocommit can we remove FacetLabel here?
   public void add(String field, FacetLabel... paths) {
     add(field, Constants.DEFAULT_DELIM_CHAR, paths);
   }
 
-  // nocommit can we remove CatPath here?
+  // nocommit can we remove FacetLabel here?
   public void add(String field, char delimChar, FacetLabel... paths) {
     Query q;
     if (paths[0].length == 0) {

Copied: lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillSideways.java (from r1542024, lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillSideways.java?p2=lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillSideways.java&p1=lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java&r1=1542024&r2=1542713&rev=1542713&view=diff
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java (original)
+++ lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillSideways.java Sun Nov 17 12:28:18 2013
@@ -1,4 +1,4 @@
-package org.apache.lucene.facet.search;
+package org.apache.lucene.facet.simple;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -19,6 +19,7 @@ package org.apache.lucene.facet.search;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -27,6 +28,7 @@ import java.util.Set;
 
 import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.params.FacetSearchParams;
+import org.apache.lucene.facet.search.DrillDownQuery;
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetFields;
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
@@ -69,94 +71,56 @@ import org.apache.lucene.search.Weight;
  * @lucene.experimental
  */
 
-public class DrillSideways {
+public class SimpleDrillSideways {
 
   protected final IndexSearcher searcher;
   protected final TaxonomyReader taxoReader;
   protected final SortedSetDocValuesReaderState state;
-  
+  protected final FacetsConfig facetsConfig;
+
   /**
    * Create a new {@code DrillSideways} instance, assuming the categories were
    * indexed with {@link FacetFields}.
    */
-  public DrillSideways(IndexSearcher searcher, TaxonomyReader taxoReader) {
-    this.searcher = searcher;
-    this.taxoReader = taxoReader;
-    this.state = null;
+  public SimpleDrillSideways(IndexSearcher searcher, FacetsConfig facetsConfig, TaxonomyReader taxoReader) {
+    this(searcher, facetsConfig, taxoReader, null);
   }
-  
+    
   /**
    * Create a new {@code DrillSideways} instance, assuming the categories were
    * indexed with {@link SortedSetDocValuesFacetFields}.
    */
-  public DrillSideways(IndexSearcher searcher, SortedSetDocValuesReaderState state) {
+  public SimpleDrillSideways(IndexSearcher searcher, FacetsConfig facetsConfig, SortedSetDocValuesReaderState state) {
+    this(searcher, facetsConfig, null, state);
+  }
+
+  /**
+   * Create a new {@code DrillSideways} instance, where some
+   * dimensions are sorted set facets and others are
+   * taxononmy facets.
+   */
+  public SimpleDrillSideways(IndexSearcher searcher, FacetsConfig facetsConfig, TaxonomyReader taxoReader, SortedSetDocValuesReaderState state) {
     this.searcher = searcher;
-    this.taxoReader = null;
+    this.facetsConfig = facetsConfig;
+    this.taxoReader = taxoReader;
     this.state = state;
   }
 
-  /** Moves any drill-downs that don't have a corresponding
-   *  facet request into the baseQuery.  This is unusual,
-   *  yet allowed, because typically the added drill-downs are because
-   *  the user has clicked on previously presented facets,
-   *  and those same facets would be computed this time
-   *  around. */
-  private static DrillDownQuery moveDrillDownOnlyClauses(DrillDownQuery in, FacetSearchParams fsp) {
-    Set<String> facetDims = new HashSet<String>();
-    for(FacetRequest fr : fsp.facetRequests) {
-      if (fr.categoryPath.length == 0) {
-        throw new IllegalArgumentException("all FacetRequests must have CategoryPath with length > 0");
-      }
-      facetDims.add(fr.categoryPath.components[0]);
-    }
+  /** Subclass can override to customize per-dim Facets
+   *  impl. */
+  protected Facets buildFacetsResult(SimpleFacetsCollector drillDowns, SimpleFacetsCollector[] drillSideways, String[] drillSidewaysDims) throws IOException {
 
-    BooleanClause[] clauses = in.getBooleanQuery().getClauses();
-    Map<String,Integer> drillDownDims = in.getDims();
+    Facets drillDownFacets = new TaxonomyFacetCounts(taxoReader, facetsConfig, drillDowns);
 
-    String[] dimsByIndex = new String[drillDownDims.size()];
-    for(Map.Entry<String,Integer> ent : drillDownDims.entrySet()) {
-      dimsByIndex[ent.getValue()] = ent.getKey();
-    }
-
-    int startClause;
-    if (clauses.length == drillDownDims.size()) {
-      startClause = 0;
+    if (drillSideways == null) {
+      return drillDownFacets;
     } else {
-      assert clauses.length == 1+drillDownDims.size();
-      startClause = 1;
-    }
-
-    // Break out drill-down clauses that have no
-    // corresponding facet request and move them inside the
-    // baseQuery:
-    List<Query> nonFacetClauses = new ArrayList<Query>();
-    List<Query> facetClauses = new ArrayList<Query>();
-    Map<String,Integer> dimToIndex = new LinkedHashMap<String,Integer>();
-    for(int i=startClause;i<clauses.length;i++) {
-      Query q = clauses[i].getQuery();
-      String dim = dimsByIndex[i-startClause];
-      if (!facetDims.contains(dim)) {
-        nonFacetClauses.add(q);
-      } else {
-        facetClauses.add(q);
-        dimToIndex.put(dim, dimToIndex.size());
-      }
-    }
-
-    if (!nonFacetClauses.isEmpty()) {
-      BooleanQuery newBaseQuery = new BooleanQuery(true);
-      if (startClause == 1) {
-        // Add original basaeQuery:
-        newBaseQuery.add(clauses[0].getQuery(), BooleanClause.Occur.MUST);
+      Map<String,Facets> drillSidewaysFacets = new HashMap<String,Facets>();
+      for(int i=0;i<drillSideways.length;i++) {
+        drillSidewaysFacets.put(drillSidewaysDims[i],
+                                new TaxonomyFacetCounts(taxoReader, facetsConfig, drillSideways[i]));
       }
-      for(Query q : nonFacetClauses) {
-        newBaseQuery.add(q, BooleanClause.Occur.MUST);
-      }
-
-      return new DrillDownQuery(fsp.indexingParams, newBaseQuery, facetClauses, dimToIndex);
-    } else {
-      // No change:
-      return in;
+      return new MultiFacets(drillSidewaysFacets, drillDownFacets);
     }
   }
 
@@ -165,36 +129,17 @@ public class DrillSideways {
    * computing drill down and sideways counts.
    */
   @SuppressWarnings({"rawtypes","unchecked"})
-  public DrillSidewaysResult search(DrillDownQuery query,
-                                    Collector hitCollector, FacetSearchParams fsp) throws IOException {
-
-    if (query.fip != fsp.indexingParams) {
-      throw new IllegalArgumentException("DrillDownQuery's FacetIndexingParams should match FacetSearchParams'");
-    }
-
-    query = moveDrillDownOnlyClauses(query, fsp);
+  public SimpleDrillSidewaysResult search(SimpleDrillDownQuery query, Collector hitCollector) throws IOException {
 
     Map<String,Integer> drillDownDims = query.getDims();
 
+    SimpleFacetsCollector drillDownCollector = new SimpleFacetsCollector();
+    
     if (drillDownDims.isEmpty()) {
-      // Just do ordinary search when there are no drill-downs:
-      FacetsCollector c = FacetsCollector.create(getDrillDownAccumulator(fsp));
-      searcher.search(query, MultiCollector.wrap(hitCollector, c));
-      return new DrillSidewaysResult(c.getFacetResults(), null);      
-    }
-
-    List<FacetRequest> ddRequests = new ArrayList<FacetRequest>();
-    for(FacetRequest fr : fsp.facetRequests) {
-      assert fr.categoryPath.length > 0;
-      if (!drillDownDims.containsKey(fr.categoryPath.components[0])) {
-        ddRequests.add(fr);
-      }
-    }
-    FacetSearchParams fsp2;
-    if (!ddRequests.isEmpty()) {
-      fsp2 = new FacetSearchParams(fsp.indexingParams, ddRequests);
-    } else {
-      fsp2 = null;
+      // There are no drill-down dims, so there is no
+      // drill-sideways to compute:
+      searcher.search(query, MultiCollector.wrap(hitCollector, drillDownCollector));
+      return new SimpleDrillSidewaysResult(buildFacetsResult(drillDownCollector, null, null), null);
     }
 
     BooleanQuery ddq = query.getBooleanQuery();
@@ -213,23 +158,11 @@ public class DrillSideways {
       startClause = 1;
     }
 
-    FacetsCollector drillDownCollector = fsp2 == null ? null : FacetsCollector.create(getDrillDownAccumulator(fsp2));
-
-    FacetsCollector[] drillSidewaysCollectors = new FacetsCollector[drillDownDims.size()];
+    SimpleFacetsCollector[] drillSidewaysCollectors = new SimpleFacetsCollector[drillDownDims.size()];
 
     int idx = 0;
     for(String dim : drillDownDims.keySet()) {
-      List<FacetRequest> requests = new ArrayList<FacetRequest>();
-      for(FacetRequest fr : fsp.facetRequests) {
-        assert fr.categoryPath.length > 0;
-        if (fr.categoryPath.components[0].equals(dim)) {
-          requests.add(fr);
-        }
-      }
-      // We already moved all drill-downs that didn't have a
-      // FacetRequest, in moveDrillDownOnlyClauses above:
-      assert !requests.isEmpty();
-      drillSidewaysCollectors[idx++] = FacetsCollector.create(getDrillSidewaysAccumulator(dim, new FacetSearchParams(fsp.indexingParams, requests)));
+      drillSidewaysCollectors[idx++] = new SimpleFacetsCollector();
     }
 
     boolean useCollectorMethod = scoreSubDocsAtOnce();
@@ -278,62 +211,25 @@ public class DrillSideways {
       // continue to run "optimized"
       collectorMethod(query, baseQuery, startClause, hitCollector, drillDownCollector, drillSidewaysCollectors);
     } else {
-      DrillSidewaysQuery dsq = new DrillSidewaysQuery(baseQuery, drillDownCollector, drillSidewaysCollectors, drillDownTerms);
+      SimpleDrillSidewaysQuery dsq = new SimpleDrillSidewaysQuery(baseQuery, drillDownCollector, drillSidewaysCollectors, drillDownTerms);
       searcher.search(dsq, hitCollector);
     }
 
-    int numDims = drillDownDims.size();
-    List<FacetResult>[] drillSidewaysResults = new List[numDims];
-    List<FacetResult> drillDownResults = null;
-
-    List<FacetResult> mergedResults = new ArrayList<FacetResult>();
-    int[] requestUpto = new int[drillDownDims.size()];
-    int ddUpto = 0;
-    for(int i=0;i<fsp.facetRequests.size();i++) {
-      FacetRequest fr = fsp.facetRequests.get(i);
-      assert fr.categoryPath.length > 0;
-      Integer dimIndex = drillDownDims.get(fr.categoryPath.components[0]);
-      if (dimIndex == null) {
-        // Pure drill down dim (the current query didn't
-        // drill down on this dim):
-        if (drillDownResults == null) {
-          // Lazy init, in case all requests were against
-          // drill-sideways dims:
-          drillDownResults = drillDownCollector.getFacetResults();
-          //System.out.println("get DD results");
-        }
-        //System.out.println("add dd results " + i);
-        mergedResults.add(drillDownResults.get(ddUpto++));
-      } else {
-        // Drill sideways dim:
-        int dim = dimIndex.intValue();
-        List<FacetResult> sidewaysResult = drillSidewaysResults[dim];
-        if (sidewaysResult == null) {
-          // Lazy init, in case no facet request is against
-          // a given drill down dim:
-          sidewaysResult = drillSidewaysCollectors[dim].getFacetResults();
-          drillSidewaysResults[dim] = sidewaysResult;
-        }
-        mergedResults.add(sidewaysResult.get(requestUpto[dim]));
-        requestUpto[dim]++;
-      }
-    }
-
-    return new DrillSidewaysResult(mergedResults, null);
+    return new SimpleDrillSidewaysResult(buildFacetsResult(drillDownCollector, drillSidewaysCollectors, drillDownDims.keySet().toArray(new String[drillDownDims.size()])), null);
   }
 
   /** Uses the more general but slower method of sideways
    *  counting. This method allows an arbitrary subQuery to
    *  implement the drill down for a given dimension. */
-  private void collectorMethod(DrillDownQuery ddq, Query baseQuery, int startClause, Collector hitCollector, Collector drillDownCollector, Collector[] drillSidewaysCollectors) throws IOException {
+  private void collectorMethod(SimpleDrillDownQuery ddq, Query baseQuery, int startClause, Collector hitCollector, Collector drillDownCollector, Collector[] drillSidewaysCollectors) throws IOException {
 
     BooleanClause[] clauses = ddq.getBooleanQuery().getClauses();
 
     Map<String,Integer> drillDownDims = ddq.getDims();
 
     BooleanQuery topQuery = new BooleanQuery(true);
-    final DrillSidewaysCollector collector = new DrillSidewaysCollector(hitCollector, drillDownCollector, drillSidewaysCollectors,
-                                                                        drillDownDims);
+    final SimpleDrillSidewaysCollector collector = new SimpleDrillSidewaysCollector(hitCollector, drillDownCollector, drillSidewaysCollectors,
+                                                                                    drillDownDims);
 
     // TODO: if query is already a BQ we could copy that and
     // add clauses to it, instead of doing BQ inside BQ
@@ -413,11 +309,11 @@ public class DrillSideways {
    * Search, sorting by {@link Sort}, and computing
    * drill down and sideways counts.
    */
-  public DrillSidewaysResult search(DrillDownQuery query,
-                                    Filter filter, FieldDoc after, int topN, Sort sort, boolean doDocScores,
-                                    boolean doMaxScore, FacetSearchParams fsp) throws IOException {
+  public SimpleDrillSidewaysResult search(SimpleDrillDownQuery query,
+                                          Filter filter, FieldDoc after, int topN, Sort sort, boolean doDocScores,
+                                          boolean doMaxScore) throws IOException {
     if (filter != null) {
-      query = new DrillDownQuery(filter, query);
+      query = new SimpleDrillDownQuery(filter, query);
     }
     if (sort != null) {
       int limit = searcher.getIndexReader().maxDoc();
@@ -432,10 +328,10 @@ public class DrillSideways {
                                                                       doDocScores,
                                                                       doMaxScore,
                                                                       true);
-      DrillSidewaysResult r = search(query, hitCollector, fsp);
-      return new DrillSidewaysResult(r.facetResults, hitCollector.topDocs());
+      SimpleDrillSidewaysResult r = search(query, hitCollector);
+      return new SimpleDrillSidewaysResult(r.facets, hitCollector.topDocs());
     } else {
-      return search(after, query, topN, fsp);
+      return search(after, query, topN);
     }
   }
 
@@ -443,36 +339,16 @@ public class DrillSideways {
    * Search, sorting by score, and computing
    * drill down and sideways counts.
    */
-  public DrillSidewaysResult search(ScoreDoc after,
-                                    DrillDownQuery query, int topN, FacetSearchParams fsp) throws IOException {
+  public SimpleDrillSidewaysResult search(ScoreDoc after,
+                                          SimpleDrillDownQuery query, int topN) throws IOException {
     int limit = searcher.getIndexReader().maxDoc();
     if (limit == 0) {
       limit = 1; // the collector does not alow numHits = 0
     }
     topN = Math.min(topN, limit);
     TopScoreDocCollector hitCollector = TopScoreDocCollector.create(topN, after, true);
-    DrillSidewaysResult r = search(query, hitCollector, fsp);
-    return new DrillSidewaysResult(r.facetResults, hitCollector.topDocs());
-  }
-
-  /** Override this to use a custom drill-down {@link
-   *  FacetsAccumulator}. */
-  protected FacetsAccumulator getDrillDownAccumulator(FacetSearchParams fsp) throws IOException {
-    if (taxoReader != null) {
-      return FacetsAccumulator.create(fsp, searcher.getIndexReader(), taxoReader, null);
-    } else {
-      return FacetsAccumulator.create(fsp, state, null);
-    }
-  }
-
-  /** Override this to use a custom drill-sideways {@link
-   *  FacetsAccumulator}. */
-  protected FacetsAccumulator getDrillSidewaysAccumulator(String dim, FacetSearchParams fsp) throws IOException {
-    if (taxoReader != null) {
-      return FacetsAccumulator.create(fsp, searcher.getIndexReader(), taxoReader, null);
-    } else {
-      return FacetsAccumulator.create(fsp, state, null);
-    }
+    SimpleDrillSidewaysResult r = search(query, hitCollector);
+    return new SimpleDrillSidewaysResult(r.facets, hitCollector.topDocs());
   }
 
   /** Override this and return true if your collector
@@ -488,25 +364,18 @@ public class DrillSideways {
     return false;
   }
 
-  /**
-   * Represents the returned result from a drill sideways search. Note that if
-   * you called
-   * {@link DrillSideways#search(DrillDownQuery, Collector, FacetSearchParams)},
-   * then {@link #hits} will be {@code null}.
-   */
-  public static class DrillSidewaysResult {
+  public static class SimpleDrillSidewaysResult {
     /** Combined drill down & sideways results. */
-    public final List<FacetResult> facetResults;
+    public final Facets facets;
 
     /** Hits. */
     public final TopDocs hits;
 
-    public DrillSidewaysResult(List<FacetResult> facetResults, TopDocs hits) {
-      this.facetResults = facetResults;
+    public SimpleDrillSidewaysResult(Facets facets, TopDocs hits) {
+      this.facets = facets;
       this.hits = hits;
     }
   }
-
   private interface SetWeight {
     public void set(Weight w);
   }

Copied: lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillSidewaysCollector.java (from r1542024, lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysCollector.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillSidewaysCollector.java?p2=lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillSidewaysCollector.java&p1=lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysCollector.java&r1=1542024&r2=1542713&rev=1542713&view=diff
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysCollector.java (original)
+++ lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillSidewaysCollector.java Sun Nov 17 12:28:18 2013
@@ -1,4 +1,4 @@
-package org.apache.lucene.facet.search;
+package org.apache.lucene.facet.simple;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -32,7 +32,7 @@ import org.apache.lucene.search.Weight;
  *  passed all constraints (a true hit) or if it missed
  *  exactly one dimension (a near-miss, to count for
  *  drill-sideways counts on that dimension). */
-class DrillSidewaysCollector extends Collector {
+class SimpleDrillSidewaysCollector extends Collector {
 
   private final Collector hitCollector;
   private final Collector drillDownCollector;
@@ -48,8 +48,8 @@ class DrillSidewaysCollector extends Col
 
   private Scorer mainScorer;
 
-  public DrillSidewaysCollector(Collector hitCollector, Collector drillDownCollector, Collector[] drillSidewaysCollectors,
-                                Map<String,Integer> dims) {
+  public SimpleDrillSidewaysCollector(Collector hitCollector, Collector drillDownCollector, Collector[] drillSidewaysCollectors,
+                                      Map<String,Integer> dims) {
     this.hitCollector = hitCollector;
     this.drillDownCollector = drillDownCollector;
     this.drillSidewaysCollectors = drillSidewaysCollectors;

Copied: lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillSidewaysQuery.java (from r1542024, lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysQuery.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillSidewaysQuery.java?p2=lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillSidewaysQuery.java&p1=lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysQuery.java&r1=1542024&r2=1542713&rev=1542713&view=diff
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysQuery.java (original)
+++ lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillSidewaysQuery.java Sun Nov 17 12:28:18 2013
@@ -1,4 +1,4 @@
-package org.apache.lucene.facet.search;
+package org.apache.lucene.facet.simple;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -34,13 +34,16 @@ import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.Bits;
 
-class DrillSidewaysQuery extends Query {
+/** Only purpose is to punch through and return a
+ *  SimpleDrillSidewaysScorer */ 
+
+class SimpleDrillSidewaysQuery extends Query {
   final Query baseQuery;
   final Collector drillDownCollector;
   final Collector[] drillSidewaysCollectors;
   final Term[][] drillDownTerms;
 
-  DrillSidewaysQuery(Query baseQuery, Collector drillDownCollector, Collector[] drillSidewaysCollectors, Term[][] drillDownTerms) {
+  SimpleDrillSidewaysQuery(Query baseQuery, Collector drillDownCollector, Collector[] drillSidewaysCollectors, Term[][] drillDownTerms) {
     this.baseQuery = baseQuery;
     this.drillDownCollector = drillDownCollector;
     this.drillSidewaysCollectors = drillSidewaysCollectors;
@@ -65,7 +68,7 @@ class DrillSidewaysQuery extends Query {
     if (newQuery == baseQuery) {
       return this;
     } else {
-      return new DrillSidewaysQuery(newQuery, drillDownCollector, drillSidewaysCollectors, drillDownTerms);
+      return new SimpleDrillSidewaysQuery(newQuery, drillDownCollector, drillSidewaysCollectors, drillDownTerms);
     }
   }
   
@@ -105,12 +108,12 @@ class DrillSidewaysQuery extends Query {
       public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
                            boolean topScorer, Bits acceptDocs) throws IOException {
 
-        DrillSidewaysScorer.DocsEnumsAndFreq[] dims = new DrillSidewaysScorer.DocsEnumsAndFreq[drillDownTerms.length];
+        SimpleDrillSidewaysScorer.DocsEnumsAndFreq[] dims = new SimpleDrillSidewaysScorer.DocsEnumsAndFreq[drillDownTerms.length];
         TermsEnum termsEnum = null;
         String lastField = null;
         int nullCount = 0;
         for(int dim=0;dim<dims.length;dim++) {
-          dims[dim] = new DrillSidewaysScorer.DocsEnumsAndFreq();
+          dims[dim] = new SimpleDrillSidewaysScorer.DocsEnumsAndFreq();
           dims[dim].sidewaysCollector = drillSidewaysCollectors[dim];
           String field = drillDownTerms[dim][0].field();
           dims[dim].dim = drillDownTerms[dim][0].text();
@@ -155,9 +158,9 @@ class DrillSidewaysQuery extends Query {
           return null;
         }
 
-        return new DrillSidewaysScorer(this, context,
-                                       baseScorer,
-                                       drillDownCollector, dims);
+        return new SimpleDrillSidewaysScorer(this, context,
+                                             baseScorer,
+                                             drillDownCollector, dims);
       }
     };
   }
@@ -181,7 +184,7 @@ class DrillSidewaysQuery extends Query {
     if (this == obj) return true;
     if (!super.equals(obj)) return false;
     if (getClass() != obj.getClass()) return false;
-    DrillSidewaysQuery other = (DrillSidewaysQuery) obj;
+    SimpleDrillSidewaysQuery other = (SimpleDrillSidewaysQuery) obj;
     if (baseQuery == null) {
       if (other.baseQuery != null) return false;
     } else if (!baseQuery.equals(other.baseQuery)) return false;

Copied: lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillSidewaysScorer.java (from r1542024, lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysScorer.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillSidewaysScorer.java?p2=lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillSidewaysScorer.java&p1=lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysScorer.java&r1=1542024&r2=1542713&rev=1542713&view=diff
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysScorer.java (original)
+++ lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleDrillSidewaysScorer.java Sun Nov 17 12:28:18 2013
@@ -1,4 +1,4 @@
-package org.apache.lucene.facet.search;
+package org.apache.lucene.facet.simple;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -28,7 +28,7 @@ import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.FixedBitSet;
 
-class DrillSidewaysScorer extends Scorer {
+class SimpleDrillSidewaysScorer extends Scorer {
 
   //private static boolean DEBUG = false;
 
@@ -47,8 +47,8 @@ class DrillSidewaysScorer extends Scorer
   private int collectDocID = -1;
   private float collectScore;
 
-  DrillSidewaysScorer(Weight w, AtomicReaderContext context, Scorer baseScorer, Collector drillDownCollector,
-                      DocsEnumsAndFreq[] dims) {
+  SimpleDrillSidewaysScorer(Weight w, AtomicReaderContext context, Scorer baseScorer, Collector drillDownCollector,
+                            DocsEnumsAndFreq[] dims) {
     super(w);
     this.dims = dims;
     this.context = context;

Modified: lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleFacetResult.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleFacetResult.java?rev=1542713&r1=1542712&r2=1542713&view=diff
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleFacetResult.java (original)
+++ lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/simple/SimpleFacetResult.java Sun Nov 17 12:28:18 2013
@@ -31,6 +31,8 @@ public final class SimpleFacetResult {
 
   /** Child counts. */
   public final LabelAndValue[] labelValues;
+
+  // nocommit also return number of children?
   
   public SimpleFacetResult(FacetLabel path, Number value, LabelAndValue[] labelValues) {
     this.path = path;

Copied: lucene/dev/branches/lucene5339/lucene/facet/src/test/org/apache/lucene/facet/simple/TestSimpleDrillSideways.java (from r1542066, lucene/dev/branches/lucene5339/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/src/test/org/apache/lucene/facet/simple/TestSimpleDrillSideways.java?p2=lucene/dev/branches/lucene5339/lucene/facet/src/test/org/apache/lucene/facet/simple/TestSimpleDrillSideways.java&p1=lucene/dev/branches/lucene5339/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java&r1=1542066&r2=1542713&rev=1542713&view=diff
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java (original)
+++ lucene/dev/branches/lucene5339/lucene/facet/src/test/org/apache/lucene/facet/simple/TestSimpleDrillSideways.java Sun Nov 17 12:28:18 2013
@@ -1,4 +1,4 @@
-package org.apache.lucene.facet.search;
+package org.apache.lucene.facet.simple;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -36,7 +36,7 @@ import org.apache.lucene.facet.FacetTest
 import org.apache.lucene.facet.index.FacetFields;
 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.simple.SimpleDrillSideways.SimpleDrillSidewaysResult;
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetFields;
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState;
 import org.apache.lucene.facet.taxonomy.FacetLabel;
@@ -44,7 +44,9 @@ import org.apache.lucene.facet.taxonomy.
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
 import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
@@ -57,8 +59,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;
 import org.apache.lucene.search.SortField.Type;
+import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
@@ -71,43 +73,52 @@ import org.apache.lucene.util.InfoStream
 import org.apache.lucene.util._TestUtil;
 import org.junit.Test;
 
-public class TestDrillSideways extends FacetTestCase {
+public class TestSimpleDrillSideways extends FacetTestCase {
 
   private DirectoryTaxonomyWriter taxoWriter;
   private RandomIndexWriter writer;
   private FacetFields facetFields;
 
-  private void add(String ... categoryPaths) throws IOException {
-    Document doc = new Document();
-    List<FacetLabel> paths = new ArrayList<FacetLabel>();
-    for(String categoryPath : categoryPaths) {
-      paths.add(new FacetLabel(categoryPath, '/'));
-    }
-    facetFields.addFields(doc, paths);
-    writer.addDocument(doc);
-  }
-
   public void testBasic() throws Exception {
     Directory dir = newDirectory();
     Directory taxoDir = newDirectory();
-    writer = new RandomIndexWriter(random(), dir);
 
     // Writes facet ords to a separate directory from the
     // main index:
     taxoWriter = new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE);
 
-    // Reused across documents, to add the necessary facet
-    // fields:
-    facetFields = new FacetFields(taxoWriter);
+    FacetsConfig config = new FacetsConfig();
+    config.setHierarchical("Publish Date");
 
-    add("Author/Bob", "Publish Date/2010/10/15");
-    add("Author/Lisa", "Publish Date/2010/10/20");
-    add("Author/Lisa", "Publish Date/2012/1/1");
-    add("Author/Susan", "Publish Date/2012/1/7");
-    add("Author/Frank", "Publish Date/1999/5/5");
+    IndexWriter writer = new FacetIndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())), taxoWriter, config);
+
+    Document doc = new Document();
+    doc.add(new FacetField("Author", "Bob"));
+    doc.add(new FacetField("Publish Date", "2010", "10", "15"));
+    writer.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new FacetField("Author", "Lisa"));
+    doc.add(new FacetField("Publish Date", "2010", "10", "20"));
+    writer.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new FacetField("Author", "Lisa"));
+    doc.add(new FacetField("Publish Date", "2012", "1", "1"));
+    writer.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new FacetField("Author", "Susan"));
+    doc.add(new FacetField("Publish Date", "2012", "1", "7"));
+    writer.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new FacetField("Author", "Frank"));
+    doc.add(new FacetField("Publish Date", "1999", "5", "5"));
+    writer.addDocument(doc);
 
     // NRT open
-    IndexSearcher searcher = newSearcher(writer.getReader());
+    IndexSearcher searcher = newSearcher(DirectoryReader.open(writer, true));
     writer.close();
 
     //System.out.println("searcher=" + searcher);
@@ -116,162 +127,134 @@ public class TestDrillSideways extends F
     TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter);
     taxoWriter.close();
 
-    // Count both "Publish Date" and "Author" dimensions, in
-    // drill-down:
-    FacetSearchParams fsp = new FacetSearchParams(
-        new CountFacetRequest(new FacetLabel("Publish Date"), 10), 
-        new CountFacetRequest(new FacetLabel("Author"), 10));
-
-    DrillSideways ds = new DrillSideways(searcher, taxoReader);
+    SimpleDrillSideways ds = new SimpleDrillSideways(searcher, config, taxoReader);
 
     // Simple case: drill-down on a single field; in this
     // case the drill-sideways + drill-down counts ==
     // drill-down of just the query: 
-    DrillDownQuery ddq = new DrillDownQuery(fsp.indexingParams, new MatchAllDocsQuery());
+    SimpleDrillDownQuery ddq = new SimpleDrillDownQuery();
     ddq.add(new FacetLabel("Author", "Lisa"));
-    DrillSidewaysResult r = ds.search(null, ddq, 10, fsp);
-
+    SimpleDrillSidewaysResult r = ds.search(null, ddq, 10);
     assertEquals(2, r.hits.totalHits);
-    assertEquals(2, r.facetResults.size());
     // Publish Date is only drill-down, and Lisa published
     // one in 2012 and one in 2010:
-    assertEquals("Publish Date: 2012=1 2010=1", toString(r.facetResults.get(0)));
+    assertEquals("Publish Date (2)\n  2010 (1)\n  2012 (1)\n", r.facets.getTopChildren(10, "Publish Date").toString());
+
     // Author is drill-sideways + drill-down: Lisa
     // (drill-down) published twice, and Frank/Susan/Bob
     // published once:
-    assertEquals("Author: Lisa=2 Frank=1 Susan=1 Bob=1", toString(r.facetResults.get(1)));
+    assertEquals("Author (5)\n  Lisa (2)\n  Bob (1)\n  Susan (1)\n  Frank (1)\n", r.facets.getTopChildren(10, "Author").toString());
 
     // Same simple case, but no baseQuery (pure browse):
     // drill-down on a single field; in this case the
     // drill-sideways + drill-down counts == drill-down of
     // just the query:
-    ddq = new DrillDownQuery(fsp.indexingParams);
+    ddq = new SimpleDrillDownQuery();
     ddq.add(new FacetLabel("Author", "Lisa"));
-    r = ds.search(null, ddq, 10, fsp);
+    r = ds.search(null, ddq, 10);
 
     assertEquals(2, r.hits.totalHits);
-    assertEquals(2, r.facetResults.size());
     // Publish Date is only drill-down, and Lisa published
     // one in 2012 and one in 2010:
-    assertEquals("Publish Date: 2012=1 2010=1", toString(r.facetResults.get(0)));
-    assertEquals(2, r.facetResults.get(0).getNumValidDescendants());
+    assertEquals("Publish Date (2)\n  2010 (1)\n  2012 (1)\n", r.facets.getTopChildren(10, "Publish Date").toString());
 
     // Author is drill-sideways + drill-down: Lisa
     // (drill-down) published twice, and Frank/Susan/Bob
     // published once:
-    assertEquals("Author: Lisa=2 Frank=1 Susan=1 Bob=1", toString(r.facetResults.get(1)));
-    assertEquals(4, r.facetResults.get(1).getNumValidDescendants());
+    assertEquals("Author (5)\n  Lisa (2)\n  Bob (1)\n  Susan (1)\n  Frank (1)\n", r.facets.getTopChildren(10, "Author").toString());
 
     // Another simple case: drill-down on on single fields
     // but OR of two values
-    ddq = new DrillDownQuery(fsp.indexingParams, new MatchAllDocsQuery());
+    ddq = new SimpleDrillDownQuery();
     ddq.add(new FacetLabel("Author", "Lisa"), new FacetLabel("Author", "Bob"));
-    r = ds.search(null, ddq, 10, fsp);
+    r = ds.search(null, ddq, 10);
     assertEquals(3, r.hits.totalHits);
-    assertEquals(2, r.facetResults.size());
     // Publish Date is only drill-down: Lisa and Bob
     // (drill-down) published twice in 2010 and once in 2012:
-    assertEquals("Publish Date: 2010=2 2012=1", toString(r.facetResults.get(0)));
+    assertEquals("Publish Date (3)\n  2010 (2)\n  2012 (1)\n", r.facets.getTopChildren(10, "Publish Date").toString());
     // Author is drill-sideways + drill-down: Lisa
     // (drill-down) published twice, and Frank/Susan/Bob
     // published once:
-    assertEquals("Author: Lisa=2 Frank=1 Susan=1 Bob=1", toString(r.facetResults.get(1)));
+    assertEquals("Author (5)\n  Lisa (2)\n  Bob (1)\n  Susan (1)\n  Frank (1)\n", r.facets.getTopChildren(10, "Author").toString());
 
     // More interesting case: drill-down on two fields
-    ddq = new DrillDownQuery(fsp.indexingParams, new MatchAllDocsQuery());
+    ddq = new SimpleDrillDownQuery();
     ddq.add(new FacetLabel("Author", "Lisa"));
     ddq.add(new FacetLabel("Publish Date", "2010"));
-    r = ds.search(null, ddq, 10, fsp);
+    r = ds.search(null, ddq, 10);
     assertEquals(1, r.hits.totalHits);
-    assertEquals(2, r.facetResults.size());
     // Publish Date is drill-sideways + drill-down: Lisa
     // (drill-down) published once in 2010 and once in 2012:
-    assertEquals("Publish Date: 2012=1 2010=1", toString(r.facetResults.get(0)));
+    assertEquals("Publish Date (2)\n  2010 (1)\n  2012 (1)\n", r.facets.getTopChildren(10, "Publish Date").toString());
     // Author is drill-sideways + drill-down:
     // only Lisa & Bob published (once each) in 2010:
-    assertEquals("Author: Lisa=1 Bob=1", toString(r.facetResults.get(1)));
+    assertEquals("Author (2)\n  Bob (1)\n  Lisa (1)\n", r.facets.getTopChildren(10, "Author").toString());
 
     // Even more interesting case: drill down on two fields,
     // but one of them is OR
-    ddq = new DrillDownQuery(fsp.indexingParams, new MatchAllDocsQuery());
+    ddq = new SimpleDrillDownQuery();
 
     // Drill down on Lisa or Bob:
     ddq.add(new FacetLabel("Author", "Lisa"),
             new FacetLabel("Author", "Bob"));
     ddq.add(new FacetLabel("Publish Date", "2010"));
-    r = ds.search(null, ddq, 10, fsp);
+    r = ds.search(null, ddq, 10);
     assertEquals(2, r.hits.totalHits);
-    assertEquals(2, r.facetResults.size());
     // Publish Date is both drill-sideways + drill-down:
     // Lisa or Bob published twice in 2010 and once in 2012:
-    assertEquals("Publish Date: 2010=2 2012=1", toString(r.facetResults.get(0)));
+    assertEquals("Publish Date (3)\n  2010 (2)\n  2012 (1)\n", r.facets.getTopChildren(10, "Publish Date").toString());
     // Author is drill-sideways + drill-down:
     // only Lisa & Bob published (once each) in 2010:
-    assertEquals("Author: Lisa=1 Bob=1", toString(r.facetResults.get(1)));
+    assertEquals("Author (2)\n  Bob (1)\n  Lisa (1)\n", r.facets.getTopChildren(10, "Author").toString());
 
     // Test drilling down on invalid field:
-    ddq = new DrillDownQuery(fsp.indexingParams, new MatchAllDocsQuery());
+    ddq = new SimpleDrillDownQuery();
     ddq.add(new FacetLabel("Foobar", "Baz"));
-    fsp = new FacetSearchParams(
-        new CountFacetRequest(new FacetLabel("Publish Date"), 10), 
-        new CountFacetRequest(new FacetLabel("Foobar"), 10));
-    r = ds.search(null, ddq, 10, fsp);
+    r = ds.search(null, ddq, 10);
     assertEquals(0, r.hits.totalHits);
-    assertEquals(2, r.facetResults.size());
-    assertEquals("Publish Date:", toString(r.facetResults.get(0)));
-    assertEquals("Foobar:", toString(r.facetResults.get(1)));
+    assertNull(r.facets.getTopChildren(10, "Publish Date"));
+    assertNull(r.facets.getTopChildren(10, "Foobar"));
 
     // Test drilling down on valid term or'd with invalid term:
-    ddq = new DrillDownQuery(fsp.indexingParams, new MatchAllDocsQuery());
+    ddq = new SimpleDrillDownQuery();
     ddq.add(new FacetLabel("Author", "Lisa"),
             new FacetLabel("Author", "Tom"));
-    fsp = new FacetSearchParams(
-        new CountFacetRequest(new FacetLabel("Publish Date"), 10), 
-        new CountFacetRequest(new FacetLabel("Author"), 10));
-    r = ds.search(null, ddq, 10, fsp);
+    r = ds.search(null, ddq, 10);
     assertEquals(2, r.hits.totalHits);
-    assertEquals(2, r.facetResults.size());
     // Publish Date is only drill-down, and Lisa published
     // one in 2012 and one in 2010:
-    assertEquals("Publish Date: 2012=1 2010=1", toString(r.facetResults.get(0)));
+    assertEquals("Publish Date (2)\n  2010 (1)\n  2012 (1)\n", r.facets.getTopChildren(10, "Publish Date").toString());
     // Author is drill-sideways + drill-down: Lisa
     // (drill-down) published twice, and Frank/Susan/Bob
     // published once:
-    assertEquals("Author: Lisa=2 Frank=1 Susan=1 Bob=1", toString(r.facetResults.get(1)));
+    assertEquals("Author (5)\n  Lisa (2)\n  Bob (1)\n  Susan (1)\n  Frank (1)\n", r.facets.getTopChildren(10, "Author").toString());
 
     // LUCENE-4915: test drilling down on a dimension but
     // NOT facet counting it:
-    ddq = new DrillDownQuery(fsp.indexingParams, new MatchAllDocsQuery());
+    ddq = new SimpleDrillDownQuery();
     ddq.add(new FacetLabel("Author", "Lisa"),
             new FacetLabel("Author", "Tom"));
-    fsp = new FacetSearchParams(
-              new CountFacetRequest(new FacetLabel("Publish Date"), 10));
-    r = ds.search(null, ddq, 10, fsp);
+    r = ds.search(null, ddq, 10);
     assertEquals(2, r.hits.totalHits);
-    assertEquals(1, r.facetResults.size());
     // Publish Date is only drill-down, and Lisa published
     // one in 2012 and one in 2010:
-    assertEquals("Publish Date: 2012=1 2010=1", toString(r.facetResults.get(0)));
+    assertEquals("Publish Date (2)\n  2010 (1)\n  2012 (1)\n", r.facets.getTopChildren(10, "Publish Date").toString());
 
     // Test main query gets null scorer:
-    fsp = new FacetSearchParams(
-        new CountFacetRequest(new FacetLabel("Publish Date"), 10), 
-        new CountFacetRequest(new FacetLabel("Author"), 10));
-    ddq = new DrillDownQuery(fsp.indexingParams, new TermQuery(new Term("foobar", "baz")));
+    ddq = new SimpleDrillDownQuery(new TermQuery(new Term("foobar", "baz")));
     ddq.add(new FacetLabel("Author", "Lisa"));
-    r = ds.search(null, ddq, 10, fsp);
+    r = ds.search(null, ddq, 10);
 
     assertEquals(0, r.hits.totalHits);
-    assertEquals(2, r.facetResults.size());
-    assertEquals("Publish Date:", toString(r.facetResults.get(0)));
-    assertEquals("Author:", toString(r.facetResults.get(1)));
-
+    assertNull(r.facets.getTopChildren(10, "Publish Date"));
+    assertNull(r.facets.getTopChildren(10, "Author"));
     searcher.getIndexReader().close();
     taxoReader.close();
     dir.close();
     taxoDir.close();
   }
 
+  /*
   public void testSometimesInvalidDrillDown() throws Exception {
     Directory dir = newDirectory();
     Directory taxoDir = newDirectory();
@@ -439,11 +422,9 @@ public class TestDrillSideways extends F
     while (cChance == 0.0) {
       cChance = random().nextDouble();
     }
-    /*
-    aChance = .01;
-    bChance = 0.5;
-    cChance = 1.0;
-    */
+    //aChance = .01;
+    //bChance = 0.5;
+    //cChance = 1.0;
     double sum = aChance + bChance + cChance;
     aChance /= sum;
     bChance /= sum;
@@ -1124,7 +1105,7 @@ public class TestDrillSideways extends F
     }
   }
 
-  /** Just gathers counts of values under the dim. */
+  / ** Just gathers counts of values under the dim. * /
   private String toString(FacetResult fr) {
     StringBuilder b = new StringBuilder();
     FacetResultNode node = fr.getFacetResultNode();
@@ -1166,5 +1147,6 @@ public class TestDrillSideways extends F
     
     IOUtils.close(searcher.getIndexReader(), taxoReader, dir, taxoDir);
   }
+  */
 }
 

Modified: lucene/dev/branches/lucene5339/lucene/facet/src/test/org/apache/lucene/facet/simple/TestTaxonomyFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/src/test/org/apache/lucene/facet/simple/TestTaxonomyFacets.java?rev=1542713&r1=1542712&r2=1542713&view=diff
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/src/test/org/apache/lucene/facet/simple/TestTaxonomyFacets.java (original)
+++ lucene/dev/branches/lucene5339/lucene/facet/src/test/org/apache/lucene/facet/simple/TestTaxonomyFacets.java Sun Nov 17 12:28:18 2013
@@ -65,8 +65,6 @@ public class TestTaxonomyFacets extends 
 
     IndexWriter writer = new FacetIndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())), taxoWriter, fts);
 
-    // Reused across documents, to add the necessary facet
-    // fields:
     Document doc = new Document();
     doc.add(new FacetField("Author", "Bob"));
     doc.add(new FacetField("Publish Date", "2010", "10", "15"));



Mime
View raw message