You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2013/02/26 01:02:17 UTC

svn commit: r1449972 - in /lucene/dev/trunk/lucene: ./ core/src/java/org/apache/lucene/search/ core/src/java/org/apache/lucene/util/ facet/src/java/org/apache/lucene/facet/search/ facet/src/test/org/apache/lucene/facet/search/ test-framework/src/java/o...

Author: mikemccand
Date: Tue Feb 26 00:02:16 2013
New Revision: 1449972

URL: http://svn.apache.org/r1449972
Log:
LUCENE-4748: add DrillSideways utility class to facets module

Added:
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java   (with props)
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysQuery.java   (with props)
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysScorer.java   (with props)
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DrillDownQuery.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKInEachNodeResultHandler.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKResultsHandler.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/AssertingIndexSearcher.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1449972&r1=1449971&r2=1449972&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Tue Feb 26 00:02:16 2013
@@ -76,6 +76,9 @@ Changes in backwards compatibility polic
   should override FacetsAccumualtor and return the relevant aggregator,
   for aggregating the association values. (Shai Erera)
   
+* LUCENE-4748: A FacetRequest on a non-existent field now returns an
+  empty FacetResult instead of skipping it.  (Shai Erera, Mike McCandless)
+
 Optimizations
 
 * LUCENE-4687: BloomFilterPostingsFormat now lazily initializes delegate
@@ -179,6 +182,10 @@ New Features
 
 * LUCENE-4780: Add MonotonicAppendingLongBuffer: an append-only buffer for
   monotonically increasing values.  (Adrien Grand)
+
+* LUCENE-4748: Added DrillSideways utility class for computing both
+  drill-down and drill-sideways counts for a DrillDownQuery.  (Mike
+  McCandless)
  
 API Changes
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1449972&r1=1449971&r2=1449972&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java Tue Feb 26 00:02:16 2013
@@ -516,7 +516,7 @@ public class FilteredQuery extends Query
       }
       
       final Bits filterAcceptDocs = docIdSet.bits();
-        // force if RA is requested
+      // force if RA is requested
       final boolean useRandomAccess = (filterAcceptDocs != null && (useRandomAccess(filterAcceptDocs, firstFilterDoc)));
       if (useRandomAccess) {
         // if we are using random access, we return the inner scorer, just with other acceptDocs

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java?rev=1449972&r1=1449971&r2=1449972&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java Tue Feb 26 00:02:16 2013
@@ -118,7 +118,7 @@ public final class FixedBitSet extends D
   }
 
   public void set(int index) {
-    assert index >= 0 && index < numBits;
+    assert index >= 0 && index < numBits: "index=" + index + " numBits=" + numBits;
     int wordNum = index >> 6;      // div 64
     int bit = index & 0x3f;     // mod 64
     long bitmask = 1L << bit;

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DrillDownQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DrillDownQuery.java?rev=1449972&r1=1449971&r2=1449972&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DrillDownQuery.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DrillDownQuery.java Tue Feb 26 00:02:16 2013
@@ -18,8 +18,8 @@ package org.apache.lucene.facet.search;
  */
 
 import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
+import java.util.LinkedHashMap;
+import java.util.Map;
 
 import org.apache.lucene.facet.params.CategoryListParams;
 import org.apache.lucene.facet.params.FacetIndexingParams;
@@ -27,8 +27,11 @@ import org.apache.lucene.facet.taxonomy.
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.FilteredQuery;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
@@ -49,7 +52,7 @@ import org.apache.lucene.search.TermQuer
 public final class DrillDownQuery extends Query {
 
   /** Return a drill-down {@link Term} for a category. */
-  public static final Term term(FacetIndexingParams iParams, CategoryPath path) {
+  public static Term term(FacetIndexingParams iParams, CategoryPath path) {
     CategoryListParams clp = iParams.getCategoryListParams(path);
     char[] buffer = new char[path.fullPathLength()];
     iParams.drillDownTermText(path, buffer);
@@ -57,21 +60,37 @@ public final class DrillDownQuery extend
   }
   
   private final BooleanQuery query;
-  private final Set<String> drillDownDims = new HashSet<String>();
-
+  private final Map<String,Integer> drillDownDims = new LinkedHashMap<String,Integer>();
   private final FacetIndexingParams fip;
 
-  /* Used by clone() */
-  private DrillDownQuery(FacetIndexingParams fip, BooleanQuery query, Set<String> drillDownDims) {
+  /** Used by clone() */
+  DrillDownQuery(FacetIndexingParams fip, BooleanQuery query, Map<String,Integer> drillDownDims) {
     this.fip = fip;
     this.query = query.clone();
-    this.drillDownDims.addAll(drillDownDims);
+    this.drillDownDims.putAll(drillDownDims);
+  }
+
+  /** Used by DrillSideways */
+  DrillDownQuery(Filter filter, DrillDownQuery other) {
+    query = new BooleanQuery(true); // disable coord
+
+    BooleanClause[] clauses = other.query.getClauses();
+    if (clauses.length == other.drillDownDims.size()) {
+      throw new IllegalArgumentException("cannot apply filter unless baseQuery isn't null; pass ConstantScoreQuery instead");
+    }
+    assert clauses.length == 1+other.drillDownDims.size(): clauses.length + " vs " + (1+other.drillDownDims.size());
+    drillDownDims.putAll(other.drillDownDims);
+    query.add(new FilteredQuery(clauses[0].getQuery(), filter), Occur.MUST);
+    for(int i=1;i<clauses.length;i++) {
+      query.add(clauses[i].getQuery(), Occur.MUST);
+    }
+    fip = other.fip;
   }
 
   /**
-   * Creates a new {@link DrillDownQuery} without a base query, which means that
-   * you intend to perfor a pure browsing query (equivalent to using
-   * {@link MatchAllDocsQuery} as base.
+   * Creates a new {@link DrillDownQuery} without a base query, 
+   * to perform a pure browsing query (equivalent to using
+   * {@link MatchAllDocsQuery} as base).
    */
   public DrillDownQuery(FacetIndexingParams fip) {
     this(fip, null);
@@ -97,14 +116,14 @@ public final class DrillDownQuery extend
    */
   public void add(CategoryPath... paths) {
     Query q;
+    if (paths[0].length == 0) {
+      throw new IllegalArgumentException("all CategoryPaths must have length > 0");
+    }
     String dim = paths[0].components[0];
-    if (drillDownDims.contains(dim)) {
+    if (drillDownDims.containsKey(dim)) {
       throw new IllegalArgumentException("dimension '" + dim + "' was already added");
     }
     if (paths.length == 1) {
-      if (paths[0].length == 0) {
-        throw new IllegalArgumentException("all CategoryPaths must have length > 0");
-      }
       q = new TermQuery(term(fip, paths[0]));
     } else {
       BooleanQuery bq = new BooleanQuery(true); // disable coord
@@ -120,7 +139,7 @@ public final class DrillDownQuery extend
       }
       q = bq;
     }
-    drillDownDims.add(dim);
+    drillDownDims.put(dim, drillDownDims.size());
 
     final ConstantScoreQuery drillDownQuery = new ConstantScoreQuery(q);
     drillDownQuery.setBoost(0.0f);
@@ -162,5 +181,12 @@ public final class DrillDownQuery extend
   public String toString(String field) {
     return query.toString(field);
   }
-  
+
+  BooleanQuery getBooleanQuery() {
+    return query;
+  }
+
+  Map<String,Integer> getDims() {
+    return drillDownDims;
+  }
 }

Added: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java?rev=1449972&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java (added)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java Tue Feb 26 00:02:16 2013
@@ -0,0 +1,242 @@
+package org.apache.lucene.facet.search;
+
+/*
+ * 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.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.facet.params.FacetSearchParams;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.FieldDoc;
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TopFieldCollector;
+import org.apache.lucene.search.TopScoreDocCollector;
+
+/**     
+ * Computes drill down and sideways counts for the provided
+ * {@link DrillDownQuery}.  Drill sideways counts include
+ * alternative values/aggregates for the drill-down
+ * dimensions so that a dimension does not disappear after
+ * the user drills down into it.
+ *
+ * <p> Use one of the static search
+ * methods to do the search, and then get the hits and facet
+ * results from the returned {@link DrillSidewaysResult}.
+ *
+ * <p><b>NOTE</b>: this allocates one {@link
+ * FacetsCollector} for each drill-down, plus one.  If your
+ * index has high number of facet labels then this will
+ * multiply your memory usage.
+ *
+ * @lucene.experimental
+ */
+
+public class DrillSideways {
+
+  protected final IndexSearcher searcher;
+  protected final TaxonomyReader taxoReader;
+
+  /** Create a new {@code DrillSideways} instance. */
+  public DrillSideways(IndexSearcher searcher, TaxonomyReader taxoReader) {
+    this.searcher = searcher;
+    this.taxoReader = taxoReader;
+  }
+
+  /**
+   * Search, collecting hits with a {@link Collector}, and
+   * computing drill down and sideways counts.
+   */
+  public DrillSidewaysResult search(DrillDownQuery query,
+                                    Collector hitCollector, FacetSearchParams fsp) throws IOException {
+
+    Map<String,Integer> drillDownDims = query.getDims();
+
+    if (drillDownDims.isEmpty()) {
+      throw new IllegalArgumentException("there must be at least one drill-down");
+    }
+
+    BooleanQuery ddq = query.getBooleanQuery();
+    BooleanClause[] clauses = ddq.getClauses();
+
+    for(FacetRequest fr :  fsp.facetRequests) {
+      if (fr.categoryPath.length == 0) {
+        throw new IllegalArgumentException("all FacetRequests must have CategoryPath with length > 0");
+      }
+    }
+
+    Query baseQuery;
+    int startClause;
+    if (clauses.length == drillDownDims.size()) {
+      // TODO: we could optimize this pure-browse case by
+      // making a custom scorer instead:
+      baseQuery = new MatchAllDocsQuery();
+      startClause = 0;
+    } else {
+      assert clauses.length == 1+drillDownDims.size();
+      baseQuery = clauses[0].getQuery();
+      startClause = 1;
+    }
+
+    Term[][] drillDownTerms = new Term[clauses.length-startClause][];
+    for(int i=startClause;i<clauses.length;i++) {
+      Query q = clauses[i].getQuery();
+      assert q instanceof ConstantScoreQuery;
+      q = ((ConstantScoreQuery) q).getQuery();
+      assert q instanceof TermQuery || q instanceof BooleanQuery;
+      if (q instanceof TermQuery) {
+        drillDownTerms[i-startClause] = new Term[] {((TermQuery) q).getTerm()};
+      } else {
+        BooleanQuery q2 = (BooleanQuery) q;
+        BooleanClause[] clauses2 = q2.getClauses();
+        drillDownTerms[i-startClause] = new Term[clauses2.length];
+        for(int j=0;j<clauses2.length;j++) {
+          assert clauses2[j].getQuery() instanceof TermQuery;
+          drillDownTerms[i-startClause][j] = ((TermQuery) clauses2[j].getQuery()).getTerm();
+        }
+      }
+    }
+
+    FacetsCollector drillDownCollector = FacetsCollector.create(getDrillDownAccumulator(fsp));
+
+    FacetsCollector[] drillSidewaysCollectors = new FacetsCollector[drillDownDims.size()];
+
+    int idx = 0;
+    for(String dim : drillDownDims.keySet()) {
+      FacetRequest drillSidewaysRequest = null;
+      for(FacetRequest fr : fsp.facetRequests) {
+        assert fr.categoryPath.length > 0;
+        if (fr.categoryPath.components[0].equals(dim)) {
+          if (drillSidewaysRequest != null) {
+            throw new IllegalArgumentException("multiple FacetRequests for drill-sideways dimension \"" + dim + "\"");
+          }
+          drillSidewaysRequest = fr;
+        }
+      }
+      if (drillSidewaysRequest == null) {
+        throw new IllegalArgumentException("could not find FacetRequest for drill-sideways dimension \"" + dim + "\"");
+      }
+      drillSidewaysCollectors[idx++] = FacetsCollector.create(getDrillSidewaysAccumulator(dim, new FacetSearchParams(fsp.indexingParams, drillSidewaysRequest)));
+    }
+
+    DrillSidewaysQuery dsq = new DrillSidewaysQuery(baseQuery, drillDownCollector, drillSidewaysCollectors, drillDownTerms);
+
+    searcher.search(dsq, hitCollector);
+
+    List<FacetResult> drillDownResults = drillDownCollector.getFacetResults();
+
+    List<FacetResult> mergedResults = new ArrayList<FacetResult>();
+    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):
+        mergedResults.add(drillDownResults.get(i));
+      } else {
+        // Drill sideways dim:
+        List<FacetResult> sidewaysResult = drillSidewaysCollectors[dimIndex.intValue()].getFacetResults();
+
+        assert sidewaysResult.size() == 1: "size=" + sidewaysResult.size();
+        mergedResults.add(sidewaysResult.get(0));
+      }
+    }
+
+    return new DrillSidewaysResult(mergedResults, null);
+  }
+
+  /**
+   * 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 {
+    if (filter != null) {
+      query = new DrillDownQuery(filter, query);
+    }
+    if (sort != null) {
+      final TopFieldCollector hitCollector = TopFieldCollector.create(sort,
+                                                                      Math.min(topN, searcher.getIndexReader().maxDoc()),
+                                                                      after,
+                                                                      true,
+                                                                      doDocScores,
+                                                                      doMaxScore,
+                                                                      true);
+      DrillSidewaysResult r = new DrillSideways(searcher, taxoReader).search(query, hitCollector, fsp);
+      r.hits = hitCollector.topDocs();
+      return r;
+    } else {
+      return search(after, query, topN, fsp);
+    }
+  }
+
+  /**
+   * Search, sorting by score, and computing
+   * drill down and sideways counts.
+   */
+  public DrillSidewaysResult search(ScoreDoc after,
+                                    DrillDownQuery query, int topN, FacetSearchParams fsp) throws IOException {
+    TopScoreDocCollector hitCollector = TopScoreDocCollector.create(Math.min(topN, searcher.getIndexReader().maxDoc()), after, true);
+    DrillSidewaysResult r = new DrillSideways(searcher, taxoReader).search(query, hitCollector, fsp);
+    r.hits = hitCollector.topDocs();
+    return r;
+  }
+
+  /** Override this to use a custom drill-down {@link
+   *  FacetsAccumulator}. */
+  protected FacetsAccumulator getDrillDownAccumulator(FacetSearchParams fsp) {
+    return FacetsAccumulator.create(fsp, searcher.getIndexReader(), taxoReader);
+  }
+
+  /** Override this to use a custom drill-sideways {@link
+   *  FacetsAccumulator}. */
+  protected FacetsAccumulator getDrillSidewaysAccumulator(String dim, FacetSearchParams fsp) {
+    return FacetsAccumulator.create(fsp, searcher.getIndexReader(), taxoReader);
+  }
+
+  /** Represents the returned result from a drill sideways
+   *  search. */
+  public static class DrillSidewaysResult {
+    /** Combined drill down & sideways results. */
+    public final List<FacetResult> facetResults;
+
+    /** Hits. */
+    public TopDocs hits;
+
+    DrillSidewaysResult(List<FacetResult> facetResults, TopDocs hits) {
+      this.facetResults = facetResults;
+      this.hits = hits;
+    }
+  }
+}
+

Added: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysQuery.java?rev=1449972&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysQuery.java (added)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysQuery.java Tue Feb 26 00:02:16 2013
@@ -0,0 +1,169 @@
+package org.apache.lucene.facet.search;
+
+/*
+ * 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.Arrays;
+
+import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.Bits;
+
+class DrillSidewaysQuery extends Query {
+  final Query baseQuery;
+  final Collector drillDownCollector;
+  final Collector[] drillSidewaysCollectors;
+  final Term[][] drillDownTerms;
+
+  DrillSidewaysQuery(Query baseQuery, Collector drillDownCollector, Collector[] drillSidewaysCollectors, Term[][] drillDownTerms) {
+    this.baseQuery = baseQuery;
+    this.drillDownCollector = drillDownCollector;
+    this.drillSidewaysCollectors = drillSidewaysCollectors;
+    this.drillDownTerms = drillDownTerms;
+  }
+
+  @Override
+  public String toString(String field) {
+    return "DrillSidewaysQuery";
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    Query newQuery = baseQuery;
+    while(true) {
+      Query rewrittenQuery = newQuery.rewrite(reader);
+      if (rewrittenQuery == newQuery) {
+        break;
+      }
+      newQuery = rewrittenQuery;
+    }
+    if (newQuery == baseQuery) {
+      return this;
+    } else {
+      return new DrillSidewaysQuery(newQuery, drillDownCollector, drillSidewaysCollectors, drillDownTerms);
+    }
+  }
+  
+  @Override
+  public Weight createWeight(IndexSearcher searcher) throws IOException {
+    final Weight baseWeight = baseQuery.createWeight(searcher);
+
+    return new Weight() {
+      @Override
+      public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
+        return baseWeight.explain(context, doc);
+      }
+
+      @Override
+      public Query getQuery() {
+        return baseQuery;
+      }
+
+      @Override
+      public float getValueForNormalization() throws IOException {
+        return baseWeight.getValueForNormalization();
+      }
+
+      @Override
+      public void normalize(float norm, float topLevelBoost) {
+        baseWeight.normalize(norm, topLevelBoost);
+      }
+
+      @Override
+      public boolean scoresDocsOutOfOrder() {
+        // TODO: would be nice if AssertingIndexSearcher
+        // confirmed this for us
+        return false;
+      }
+
+      @Override
+      public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
+                           boolean topScorer, Bits acceptDocs) throws IOException {
+
+        DrillSidewaysScorer.DocsEnumsAndFreq[] dims = new DrillSidewaysScorer.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].sidewaysCollector = drillSidewaysCollectors[dim];
+          String field = drillDownTerms[dim][0].field();
+          dims[dim].dim = drillDownTerms[dim][0].text();
+          if (lastField == null || !lastField.equals(field)) {
+            AtomicReader reader = context.reader();
+            Terms terms = reader.terms(field);
+            if (terms != null) {
+              termsEnum = terms.iterator(null);
+            }
+            lastField = field;
+          }
+          if (termsEnum == null) {
+            nullCount++;
+            continue;
+          }
+          dims[dim].docsEnums = new DocsEnum[drillDownTerms[dim].length];
+          for(int i=0;i<drillDownTerms[dim].length;i++) {
+            if (termsEnum.seekExact(drillDownTerms[dim][i].bytes(), false)) {
+              dims[dim].freq = Math.max(dims[dim].freq, termsEnum.docFreq());
+              dims[dim].docsEnums[i] = termsEnum.docs(null, null);
+            }
+          }
+        }
+
+        if (nullCount > 1) {
+          return null;
+        }
+
+        // Sort drill-downs by most restrictive first:
+        Arrays.sort(dims);
+
+        // TODO: it could be better if we take acceptDocs
+        // into account instead of baseScorer?
+        Scorer baseScorer = baseWeight.scorer(context, scoreDocsInOrder, false, acceptDocs);
+
+        if (baseScorer == null) {
+          return null;
+        }
+
+        return new DrillSidewaysScorer(this, context,
+                                       baseScorer,
+                                       drillDownCollector, dims);
+      }
+    };
+  }
+
+  @Override
+  public int hashCode() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    throw new UnsupportedOperationException();
+  }
+}

Added: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysScorer.java?rev=1449972&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysScorer.java (added)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysScorer.java Tue Feb 26 00:02:16 2013
@@ -0,0 +1,634 @@
+package org.apache.lucene.facet.search;
+
+/*
+ * 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.Collection;
+import java.util.Collections;
+
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.FixedBitSet;
+
+class DrillSidewaysScorer extends Scorer {
+
+  //private static boolean DEBUG = false;
+
+  private final Collector drillDownCollector;
+
+  private final DocsEnumsAndFreq[] dims;
+
+  // DrillDown DocsEnums:
+  private final Scorer baseScorer;
+
+  private final AtomicReaderContext context;
+
+  private static final int CHUNK = 2048;
+  private static final int MASK = CHUNK-1;
+
+  private int collectDocID = -1;
+  private float collectScore;
+
+  DrillSidewaysScorer(Weight w, AtomicReaderContext context, Scorer baseScorer, Collector drillDownCollector,
+                      DocsEnumsAndFreq[] dims) {
+    super(w);
+    this.dims = dims;
+    this.context = context;
+    this.baseScorer = baseScorer;
+    this.drillDownCollector = drillDownCollector;
+  }
+
+  @Override
+  public void score(Collector collector) throws IOException {
+    //if (DEBUG) {
+    //  System.out.println("\nscore: reader=" + context.reader());
+    //}
+    //System.out.println("score r=" + context.reader());
+    collector.setScorer(this);
+    drillDownCollector.setScorer(this);
+    drillDownCollector.setNextReader(context);
+    for(DocsEnumsAndFreq dim : dims) {
+      dim.sidewaysCollector.setScorer(this);
+      dim.sidewaysCollector.setNextReader(context);
+    }
+
+    // TODO: if we ever allow null baseScorer ... it will
+    // mean we DO score docs out of order ... hmm, or if we
+    // change up the order of the conjuntions below
+    assert baseScorer != null;
+
+    // Position all scorers to their first matching doc:
+    int baseDocID = baseScorer.nextDoc();
+
+    for(DocsEnumsAndFreq dim : dims) {
+      for(DocsEnum docsEnum : dim.docsEnums) {
+        if (docsEnum != null) {
+          docsEnum.nextDoc();
+        }
+      }
+    }
+
+    final int numDims = dims.length;
+
+    DocsEnum[][] docsEnums = new DocsEnum[numDims][];
+    Collector[] sidewaysCollectors = new Collector[numDims];
+    int maxFreq = 0;
+    for(int dim=0;dim<numDims;dim++) {
+      docsEnums[dim] = dims[dim].docsEnums;
+      sidewaysCollectors[dim] = dims[dim].sidewaysCollector;
+      maxFreq = Math.max(maxFreq, dims[dim].freq);
+    }
+
+    // TODO: if we add cost API to Scorer, switch to that!
+    int estBaseHitCount = context.reader().maxDoc() / (1+baseDocID);
+
+    /*
+    System.out.println("\nbaseDocID=" + baseDocID + " est=" + estBaseHitCount);
+    System.out.println("  maxDoc=" + context.reader().maxDoc());
+    System.out.println("  maxFreq=" + maxFreq);
+    System.out.println("  dims[0].freq=" + dims[0].freq);
+    if (numDims > 1) {
+      System.out.println("  dims[1].freq=" + dims[1].freq);
+    }
+    */
+
+    if (estBaseHitCount < maxFreq/10) {
+      //System.out.println("baseAdvance");
+      doBaseAdvanceScoring(collector, docsEnums, sidewaysCollectors);
+    } else if (numDims > 1 && (dims[1].freq < estBaseHitCount/10)) {
+      //System.out.println("drillDownAdvance");
+      doDrillDownAdvanceScoring(collector, docsEnums, sidewaysCollectors);
+    } else {
+      //System.out.println("union");
+      doUnionScoring(collector, docsEnums, sidewaysCollectors);
+    }
+  }
+
+  /** Used when drill downs are highly constraining vs
+   *  baseQuery. */
+  private void doDrillDownAdvanceScoring(Collector collector, DocsEnum[][] docsEnums, Collector[] sidewaysCollectors) throws IOException {
+    final int maxDoc = context.reader().maxDoc();
+    final int numDims = dims.length;
+
+    //if (DEBUG) {
+    //  System.out.println("  doDrillDownAdvanceScoring");
+    //}
+
+    // TODO: maybe a class like BS, instead of parallel arrays
+    int[] filledSlots = new int[CHUNK];
+    int[] docIDs = new int[CHUNK];
+    float[] scores = new float[CHUNK];
+    int[] missingDims = new int[CHUNK];
+    int[] counts = new int[CHUNK];
+
+    docIDs[0] = -1;
+    int nextChunkStart = CHUNK;
+
+    final FixedBitSet seen = new FixedBitSet(CHUNK);
+
+    while (true) {
+      //if (DEBUG) {
+      //  System.out.println("\ncycle nextChunkStart=" + nextChunkStart + " docIds[0]=" + docIDs[0]);
+      //}
+
+      // First dim:
+      //if (DEBUG) {
+      //  System.out.println("  dim0");
+      //}
+      for(DocsEnum docsEnum : docsEnums[0]) {
+        if (docsEnum == null) {
+          continue;
+        }
+        int docID = docsEnum.docID();
+        while (docID < nextChunkStart) {
+          int slot = docID & MASK;
+
+          if (docIDs[slot] != docID) {
+            seen.set(slot);
+            // Mark slot as valid:
+            //if (DEBUG) {
+            //  System.out.println("    set docID=" + docID + " id=" + context.reader().document(docID).get("id"));
+            //}
+            docIDs[slot] = docID;
+            missingDims[slot] = 1;
+            counts[slot] = 1;
+          }
+
+          docID = docsEnum.nextDoc();
+        }
+      }
+
+      // Second dim:
+      //if (DEBUG) {
+      //  System.out.println("  dim1");
+      //}
+      for(DocsEnum docsEnum : docsEnums[1]) {
+        if (docsEnum == null) {
+          continue;
+        }
+        int docID = docsEnum.docID();
+        while (docID < nextChunkStart) {
+          int slot = docID & MASK;
+
+          if (docIDs[slot] != docID) {
+            // Mark slot as valid:
+            seen.set(slot);
+            //if (DEBUG) {
+            //  System.out.println("    set docID=" + docID + " missingDim=0 id=" + context.reader().document(docID).get("id"));
+            //}
+            docIDs[slot] = docID;
+            missingDims[slot] = 0;
+            counts[slot] = 1;
+          } else {
+            // TODO: single-valued dims will always be true
+            // below; we could somehow specialize
+            if (missingDims[slot] >= 1) {
+              missingDims[slot] = 2;
+              counts[slot] = 2;
+              //if (DEBUG) {
+              //  System.out.println("    set docID=" + docID + " missingDim=2 id=" + context.reader().document(docID).get("id"));
+              //}
+            } else {
+              counts[slot] = 1;
+              //if (DEBUG) {
+              //  System.out.println("    set docID=" + docID + " missingDim=" + missingDims[slot] + " id=" + context.reader().document(docID).get("id"));
+              //}
+            }
+          }
+
+          docID = docsEnum.nextDoc();
+        }
+      }
+
+      // After this we can "upgrade" to conjunction, because
+      // any doc not seen by either dim 0 or dim 1 cannot be
+      // a hit or a near miss:
+
+      //if (DEBUG) {
+      //  System.out.println("  baseScorer");
+      //}
+
+      // Fold in baseScorer, using advance:
+      int filledCount = 0;
+      int slot0 = 0;
+      while (slot0 < CHUNK && (slot0 = seen.nextSetBit(slot0)) != -1) {
+        int ddDocID = docIDs[slot0];
+        assert ddDocID != -1;
+
+        int baseDocID = baseScorer.docID();
+        if (baseDocID < ddDocID) {
+          baseDocID = baseScorer.advance(ddDocID);
+        }
+        if (baseDocID == ddDocID) {
+          //if (DEBUG) {
+          //  System.out.println("    keep docID=" + ddDocID + " id=" + context.reader().document(ddDocID).get("id"));
+          //}
+          scores[slot0] = baseScorer.score();
+          filledSlots[filledCount++] = slot0;
+          counts[slot0]++;
+        } else {
+          //if (DEBUG) {
+          //  System.out.println("    no docID=" + ddDocID + " id=" + context.reader().document(ddDocID).get("id"));
+          //}
+          docIDs[slot0] = -1;
+
+          // TODO: we could jump slot0 forward to the
+          // baseDocID ... but we'd need to set docIDs for
+          // intervening slots to -1
+        }
+        slot0++;
+      }
+      seen.clear(0, CHUNK);
+
+      if (filledCount == 0) {
+        if (nextChunkStart >= maxDoc) {
+          break;
+        }
+        nextChunkStart += CHUNK;
+        continue;
+      }
+      
+      // TODO: factor this out & share w/ union scorer,
+      // except we start from dim=2 instead:
+      for(int dim=2;dim<numDims;dim++) {
+        //if (DEBUG) {
+        //  System.out.println("  dim=" + dim + " [" + dims[dim].dim + "]");
+        //}
+        for(DocsEnum docsEnum : docsEnums[dim]) {
+          if (docsEnum == null) {
+            continue;
+          }
+          int docID = docsEnum.docID();
+          while (docID < nextChunkStart) {
+            int slot = docID & MASK;
+            if (docIDs[slot] == docID && counts[slot] >= dim) {
+              // TODO: single-valued dims will always be true
+              // below; we could somehow specialize
+              if (missingDims[slot] >= dim) {
+                //if (DEBUG) {
+                //  System.out.println("    set docID=" + docID + " count=" + (dim+2));
+                //}
+                missingDims[slot] = dim+1;
+                counts[slot] = dim+2;
+              } else {
+                //if (DEBUG) {
+                //  System.out.println("    set docID=" + docID + " missing count=" + (dim+1));
+                //}
+                counts[slot] = dim+1;
+              }
+            }
+            // TODO: sometimes use advance?
+            docID = docsEnum.nextDoc();
+          }
+        }
+      }
+
+      // Collect:
+      //if (DEBUG) {
+      //  System.out.println("  now collect: " + filledCount + " hits");
+      //}
+      for(int i=0;i<filledCount;i++) {
+        int slot = filledSlots[i];
+        collectDocID = docIDs[slot];
+        collectScore = scores[slot];
+        //if (DEBUG) {
+        //  System.out.println("    docID=" + docIDs[slot] + " count=" + counts[slot]);
+        //}
+        if (counts[slot] == 1+numDims) {
+          collectHit(collector, sidewaysCollectors);
+        } else if (counts[slot] == numDims) {
+          collectNearMiss(sidewaysCollectors, missingDims[slot]);
+        }
+      }
+
+      if (nextChunkStart >= maxDoc) {
+        break;
+      }
+
+      nextChunkStart += CHUNK;
+    }
+  }
+
+  /** Used when base query is highly constraining vs the
+   *  drilldowns; in this case we just .next() on base and
+   *  .advance() on the dims. */
+  private void doBaseAdvanceScoring(Collector collector, DocsEnum[][] docsEnums, Collector[] sidewaysCollectors) throws IOException {
+    //if (DEBUG) {
+    //  System.out.println("  doBaseAdvanceScoring");
+    //}
+    int docID = baseScorer.docID();
+
+    final int numDims = dims.length;
+
+    nextDoc: while (docID != NO_MORE_DOCS) {
+      int failedDim = -1;
+      for(int dim=0;dim<numDims;dim++) {
+        // TODO: should we sort this 2nd dimension of
+        // docsEnums from most frequent to least?
+        boolean found = false;
+        for(DocsEnum docsEnum : docsEnums[dim]) {
+          if (docsEnum == null) {
+            continue;
+          }
+          if (docsEnum.docID() < docID) {
+            docsEnum.advance(docID);
+          }
+          if (docsEnum.docID() == docID) {
+            found = true;
+            break;
+          }
+        }
+        if (!found) {
+          if (failedDim != -1) {
+            // More than one dim fails on this document, so
+            // it's neither a hit nor a near-miss; move to
+            // next doc:
+            docID = baseScorer.nextDoc();
+            continue nextDoc;
+          } else {
+            failedDim = dim;
+          }
+        }
+      }
+
+      collectDocID = docID;
+
+      // TODO: we could score on demand instead since we are
+      // daat here:
+      collectScore = baseScorer.score();
+
+      if (failedDim == -1) {
+        collectHit(collector, sidewaysCollectors);
+      } else {
+        collectNearMiss(sidewaysCollectors, failedDim);
+      }
+
+      docID = baseScorer.nextDoc();
+    }
+  }
+
+  private void collectHit(Collector collector, Collector[] sidewaysCollectors) throws IOException {
+    //if (DEBUG) {
+    //  System.out.println("      hit");
+    //}
+
+    collector.collect(collectDocID);
+    drillDownCollector.collect(collectDocID);
+
+    // TODO: we could "fix" faceting of the sideways counts
+    // to do this "union" (of the drill down hits) in the
+    // end instead:
+
+    // Tally sideways counts:
+    for(int dim=0;dim<sidewaysCollectors.length;dim++) {
+      sidewaysCollectors[dim].collect(collectDocID);
+    }
+  }
+
+  private void collectNearMiss(Collector[] sidewaysCollectors, int dim) throws IOException {
+    //if (DEBUG) {
+    //  System.out.println("      missingDim=" + dim);
+    //}
+    sidewaysCollectors[dim].collect(collectDocID);
+  }
+
+  private void doUnionScoring(Collector collector, DocsEnum[][] docsEnums, Collector[] sidewaysCollectors) throws IOException {
+    //if (DEBUG) {
+    //  System.out.println("  doUnionScoring");
+    //}
+
+    final int maxDoc = context.reader().maxDoc();
+    final int numDims = dims.length;
+
+    // TODO: maybe a class like BS, instead of parallel arrays
+    int[] filledSlots = new int[CHUNK];
+    int[] docIDs = new int[CHUNK];
+    float[] scores = new float[CHUNK];
+    int[] missingDims = new int[CHUNK];
+    int[] counts = new int[CHUNK];
+
+    docIDs[0] = -1;
+
+    // NOTE: this is basically a specialized version of
+    // BooleanScorer, to the minShouldMatch=N-1 case, but
+    // carefully tracking which dimension failed to match
+
+    int nextChunkStart = CHUNK;
+
+    while (true) {
+      //if (DEBUG) {
+      //  System.out.println("\ncycle nextChunkStart=" + nextChunkStart + " docIds[0]=" + docIDs[0]);
+      //}
+      int filledCount = 0;
+      int docID = baseScorer.docID();
+      //if (DEBUG) {
+      //  System.out.println("  base docID=" + docID);
+      //}
+      while (docID < nextChunkStart) {
+        int slot = docID & MASK;
+        //if (DEBUG) {
+        //  System.out.println("    docIDs[slot=" + slot + "]=" + docID + " id=" + context.reader().document(docID).get("id"));
+        //}
+
+        // Mark slot as valid:
+        assert docIDs[slot] != docID: "slot=" + slot + " docID=" + docID;
+        docIDs[slot] = docID;
+        scores[slot] = baseScorer.score();
+        filledSlots[filledCount++] = slot;
+        missingDims[slot] = 0;
+        counts[slot] = 1;
+
+        docID = baseScorer.nextDoc();
+      }
+
+      if (filledCount == 0) {
+        if (nextChunkStart >= maxDoc) {
+          break;
+        }
+        nextChunkStart += CHUNK;
+        continue;
+      }
+
+      // First drill-down dim, basically adds SHOULD onto
+      // the baseQuery:
+      //if (DEBUG) {
+      //  System.out.println("  dim=0 [" + dims[0].dim + "]");
+      //}
+      for(DocsEnum docsEnum : docsEnums[0]) {
+        if (docsEnum == null) {
+          continue;
+        }
+        docID = docsEnum.docID();
+        //if (DEBUG) {
+        //  System.out.println("    start docID=" + docID);
+        //}
+        while (docID < nextChunkStart) {
+          int slot = docID & MASK;
+          if (docIDs[slot] == docID) {
+            //if (DEBUG) {
+            //  System.out.println("      set docID=" + docID + " count=2");
+            //}
+            missingDims[slot] = 1;
+            counts[slot] = 2;
+          }
+          docID = docsEnum.nextDoc();
+        }
+      }
+
+      for(int dim=1;dim<numDims;dim++) {
+        //if (DEBUG) {
+        //  System.out.println("  dim=" + dim + " [" + dims[dim].dim + "]");
+        //}
+        for(DocsEnum docsEnum : docsEnums[dim]) {
+          if (docsEnum == null) {
+            continue;
+          }
+          docID = docsEnum.docID();
+          //if (DEBUG) {
+          //  System.out.println("    start docID=" + docID);
+          //}
+          while (docID < nextChunkStart) {
+            int slot = docID & MASK;
+            if (docIDs[slot] == docID && counts[slot] >= dim) {
+              // This doc is still in the running...
+              // TODO: single-valued dims will always be true
+              // below; we could somehow specialize
+              if (missingDims[slot] >= dim) {
+                //if (DEBUG) {
+                //  System.out.println("      set docID=" + docID + " count=" + (dim+2));
+                //}
+                missingDims[slot] = dim+1;
+                counts[slot] = dim+2;
+              } else {
+                //if (DEBUG) {
+                //  System.out.println("      set docID=" + docID + " missing count=" + (dim+1));
+                //}
+                counts[slot] = dim+1;
+              }
+            }
+            docID = docsEnum.nextDoc();
+          }
+
+          // TODO: sometimes use advance?
+
+          /*
+            int docBase = nextChunkStart - CHUNK;
+            for(int i=0;i<filledCount;i++) {
+              int slot = filledSlots[i];
+              docID = docBase + filledSlots[i];
+              if (docIDs[slot] == docID && counts[slot] >= dim) {
+                // This doc is still in the running...
+                int ddDocID = docsEnum.docID();
+                if (ddDocID < docID) {
+                  ddDocID = docsEnum.advance(docID);
+                }
+                if (ddDocID == docID) {
+                  if (missingDims[slot] >= dim && counts[slot] == allMatchCount) {
+                  //if (DEBUG) {
+                  //    System.out.println("    set docID=" + docID + " count=" + (dim+2));
+                   // }
+                    missingDims[slot] = dim+1;
+                    counts[slot] = dim+2;
+                  } else {
+                  //if (DEBUG) {
+                  //    System.out.println("    set docID=" + docID + " missing count=" + (dim+1));
+                   // }
+                    counts[slot] = dim+1;
+                  }
+                }
+              }
+            }            
+          */
+        }
+      }
+
+      // Collect:
+      //if (DEBUG) {
+      //  System.out.println("  now collect: " + filledCount + " hits");
+      //}
+      for(int i=0;i<filledCount;i++) {
+        int slot = filledSlots[i];
+        collectDocID = docIDs[slot];
+        collectScore = scores[slot];
+        //if (DEBUG) {
+        //  System.out.println("    docID=" + docIDs[slot] + " count=" + counts[slot]);
+        //}
+        //System.out.println("  collect doc=" + collectDocID + " main.freq=" + (counts[slot]-1) + " main.doc=" + collectDocID + " exactCount=" + numDims);
+        if (counts[slot] == 1+numDims) {
+          //System.out.println("    hit");
+          collectHit(collector, sidewaysCollectors);
+        } else if (counts[slot] == numDims) {
+          //System.out.println("    sw");
+          collectNearMiss(sidewaysCollectors, missingDims[slot]);
+        }
+      }
+
+      if (nextChunkStart >= maxDoc) {
+        break;
+      }
+
+      nextChunkStart += CHUNK;
+    }
+  }
+
+  @Override
+  public int docID() {
+    return collectDocID;
+  }
+
+  @Override
+  public float score() {
+    return collectScore;
+  }
+
+  @Override
+  public int freq() {
+    return 1+dims.length;
+  }
+
+  @Override
+  public int nextDoc() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int advance(int target) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Collection<ChildScorer> getChildren() {
+    return Collections.singletonList(new ChildScorer(baseScorer, "MUST"));
+  }
+
+  static class DocsEnumsAndFreq implements Comparable<DocsEnumsAndFreq> {
+    DocsEnum[] docsEnums;
+    // Max docFreq for all docsEnums for this dim:
+    int freq;
+    Collector sidewaysCollector;
+    String dim;
+
+    @Override
+    public int compareTo(DocsEnumsAndFreq other) {
+      return freq - other.freq;
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java?rev=1449972&r1=1449971&r2=1449972&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java Tue Feb 26 00:02:16 2013
@@ -60,6 +60,26 @@ public class FacetsAccumulator {
   public FacetsAccumulator(FacetSearchParams searchParams, IndexReader indexReader, TaxonomyReader taxonomyReader) {
     this(searchParams, indexReader, taxonomyReader, null);
   }
+
+  /**
+   * Creates an appropriate {@link FacetsAccumulator},
+   * returning {@link FacetsAccumulator} when all requests
+   * are {@link CountFacetRequest} and only one partition is
+   * in use, otherwise {@link StandardFacetsAccumulator}.
+   */
+  public static FacetsAccumulator create(FacetSearchParams fsp, IndexReader indexReader, TaxonomyReader taxoReader) {
+    if (fsp.indexingParams.getPartitionSize() != Integer.MAX_VALUE) {
+      return new StandardFacetsAccumulator(fsp, indexReader, taxoReader);
+    }
+    
+    for (FacetRequest fr : fsp.facetRequests) {
+      if (!(fr instanceof CountFacetRequest)) {
+        return new StandardFacetsAccumulator(fsp, indexReader, taxoReader);
+      }
+    }
+    
+    return new FacetsAccumulator(fsp, indexReader, taxoReader);
+  }
   
   /**
    * Initializes the accumulator with the given parameters as well as
@@ -153,6 +173,12 @@ public class FacetsAccumulator {
     for (FacetRequest fr : searchParams.facetRequests) {
       int rootOrd = taxonomyReader.getOrdinal(fr.categoryPath);
       if (rootOrd == TaxonomyReader.INVALID_ORDINAL) { // category does not exist
+        // Add empty FacetResult:
+        FacetResultNode root = new FacetResultNode();
+        root.ordinal = TaxonomyReader.INVALID_ORDINAL;
+        root.label = fr.categoryPath;
+        root.value = 0;
+        res.add(new FacetResult(fr, root, 0));
         continue;
       }
       CategoryListParams clp = searchParams.indexingParams.getCategoryListParams(fr.categoryPath);

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java?rev=1449972&r1=1449971&r2=1449972&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java Tue Feb 26 00:02:16 2013
@@ -163,21 +163,11 @@ public abstract class FacetsCollector ex
   }
   
   /**
-   * Creates a {@link FacetsCollector} with the default
-   * {@link FacetsAccumulator}.
+   * Creates a {@link FacetsCollector} using the {@link
+   * FacetsAccumulator} from {@link FacetsAccumulator#create}.
    */
   public static FacetsCollector create(FacetSearchParams fsp, IndexReader indexReader, TaxonomyReader taxoReader) {
-    if (fsp.indexingParams.getPartitionSize() != Integer.MAX_VALUE) {
-      return create(new StandardFacetsAccumulator(fsp, indexReader, taxoReader));
-    }
-    
-    for (FacetRequest fr : fsp.facetRequests) {
-      if (!(fr instanceof CountFacetRequest)) {
-        return create(new StandardFacetsAccumulator(fsp, indexReader, taxoReader));
-      }
-    }
-    
-    return create(new FacetsAccumulator(fsp, indexReader, taxoReader));
+    return create(FacetsAccumulator.create(fsp, indexReader, taxoReader));
   }
 
   /**

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java?rev=1449972&r1=1449971&r2=1449972&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java Tue Feb 26 00:02:16 2013
@@ -197,7 +197,13 @@ public class StandardFacetsAccumulator e
         PartitionsFacetResultsHandler frHndlr = createFacetResultsHandler(fr);
         IntermediateFacetResult tmpResult = fr2tmpRes.get(fr);
         if (tmpResult == null) {
-          continue; // do not add a null to the list.
+          // Add empty FacetResult:
+          FacetResultNode root = new FacetResultNode();
+          root.ordinal = TaxonomyReader.INVALID_ORDINAL;
+          root.label = fr.categoryPath;
+          root.value = 0;
+          res.add(new FacetResult(fr, root, 0));
+          continue;
         }
         FacetResult facetRes = frHndlr.renderFacetResult(tmpResult);
         // final labeling if allowed (because labeling is a costly operation)

Added: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java?rev=1449972&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java (added)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java Tue Feb 26 00:02:16 2013
@@ -0,0 +1,829 @@
+package org.apache.lucene.facet.search;
+
+/*
+ * 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.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+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.params.FacetSearchParams;
+import org.apache.lucene.facet.search.DrillSideways.DrillSidewaysResult;
+import org.apache.lucene.facet.taxonomy.CategoryPath;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+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.IndexReader;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+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.TermQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.InfoStream;
+import org.apache.lucene.util._TestUtil;
+
+public class TestDrillSideways extends FacetTestCase {
+
+  private DirectoryTaxonomyWriter taxoWriter;
+  private RandomIndexWriter writer;
+  private FacetFields facetFields;
+
+  private void add(String ... categoryPaths) throws IOException {
+    Document doc = new Document();
+    List<CategoryPath> paths = new ArrayList<CategoryPath>();
+    for(String categoryPath : categoryPaths) {
+      paths.add(new CategoryPath(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);
+
+    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");
+
+    // NRT open
+    IndexSearcher searcher = newSearcher(writer.getReader());
+    writer.close();
+
+    //System.out.println("searcher=" + searcher);
+
+    // NRT open
+    TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter);
+    taxoWriter.close();
+
+    // Count both "Publish Date" and "Author" dimensions, in
+    // drill-down:
+    FacetSearchParams fsp = new FacetSearchParams(
+        new CountFacetRequest(new CategoryPath("Publish Date"), 10), 
+        new CountFacetRequest(new CategoryPath("Author"), 10));
+
+    // 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());
+    ddq.add(new CategoryPath("Author", "Lisa"));
+    DrillSidewaysResult r = new DrillSideways(searcher, taxoReader).search(null, ddq, 10, fsp);
+
+    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)));
+    // 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)));
+
+    // 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.add(new CategoryPath("Author", "Lisa"));
+    r = new DrillSideways(searcher, taxoReader).search(null, ddq, 10, fsp);
+
+    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)));
+    // 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)));
+
+    // Another simple case: drill-down on on single fields
+    // but OR of two values
+    ddq = new DrillDownQuery(fsp.indexingParams, new MatchAllDocsQuery());
+    ddq.add(new CategoryPath("Author", "Lisa"), new CategoryPath("Author", "Bob"));
+    r = new DrillSideways(searcher, taxoReader).search(null, ddq, 10, fsp);
+    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)));
+    // 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)));
+
+    // More interesting case: drill-down on two fields
+    ddq = new DrillDownQuery(fsp.indexingParams, new MatchAllDocsQuery());
+    ddq.add(new CategoryPath("Author", "Lisa"));
+    ddq.add(new CategoryPath("Publish Date", "2010"));
+    r = new DrillSideways(searcher, taxoReader).search(null, ddq, 10, fsp);
+    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)));
+    // 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)));
+
+    // Even more interesting case: drill down on two fields,
+    // but one of them is OR
+    ddq = new DrillDownQuery(fsp.indexingParams, new MatchAllDocsQuery());
+
+    // Drill down on Lisa or Bob:
+    ddq.add(new CategoryPath("Author", "Lisa"),
+            new CategoryPath("Author", "Bob"));
+    ddq.add(new CategoryPath("Publish Date", "2010"));
+    r = new DrillSideways(searcher, taxoReader).search(null, ddq, 10, fsp);
+    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)));
+    // 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)));
+
+    // Test drilling down on invalid field:
+    ddq = new DrillDownQuery(fsp.indexingParams, new MatchAllDocsQuery());
+    ddq.add(new CategoryPath("Foobar", "Baz"));
+    fsp = new FacetSearchParams(
+        new CountFacetRequest(new CategoryPath("Publish Date"), 10), 
+        new CountFacetRequest(new CategoryPath("Foobar"), 10));
+    r = new DrillSideways(searcher, taxoReader).search(null, ddq, 10, fsp);
+    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)));
+
+    // Test drilling down on valid term or'd with invalid term:
+    ddq = new DrillDownQuery(fsp.indexingParams, new MatchAllDocsQuery());
+    ddq.add(new CategoryPath("Author", "Lisa"),
+            new CategoryPath("Author", "Tom"));
+    fsp = new FacetSearchParams(
+        new CountFacetRequest(new CategoryPath("Publish Date"), 10), 
+        new CountFacetRequest(new CategoryPath("Author"), 10));
+    r = new DrillSideways(searcher, taxoReader).search(null, ddq, 10, fsp);
+    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)));
+    // 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)));
+
+    // Test main query gets null scorer:
+    fsp = new FacetSearchParams(
+        new CountFacetRequest(new CategoryPath("Publish Date"), 10), 
+        new CountFacetRequest(new CategoryPath("Author"), 10));
+    ddq = new DrillDownQuery(fsp.indexingParams, new TermQuery(new Term("foobar", "baz")));
+    ddq.add(new CategoryPath("Author", "Lisa"));
+    r = new DrillSideways(searcher, taxoReader).search(null, ddq, 10, fsp);
+
+    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)));
+
+    searcher.getIndexReader().close();
+    taxoReader.close();
+    dir.close();
+    taxoDir.close();
+  }
+
+  public void testSometimesInvalidDrillDown() 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);
+
+    add("Author/Bob", "Publish Date/2010/10/15");
+    add("Author/Lisa", "Publish Date/2010/10/20");
+    writer.commit();
+    // 2nd segment has no Author:
+    add("Foobar/Lisa", "Publish Date/2012/1/1");
+
+    // NRT open
+    IndexSearcher searcher = newSearcher(writer.getReader());
+    writer.close();
+
+    //System.out.println("searcher=" + searcher);
+
+    // NRT open
+    TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter);
+    taxoWriter.close();
+
+    // Count both "Publish Date" and "Author" dimensions, in
+    // drill-down:
+    FacetSearchParams fsp = new FacetSearchParams(
+        new CountFacetRequest(new CategoryPath("Publish Date"), 10), 
+        new CountFacetRequest(new CategoryPath("Author"), 10));
+
+    DrillDownQuery ddq = new DrillDownQuery(fsp.indexingParams, new MatchAllDocsQuery());
+    ddq.add(new CategoryPath("Author", "Lisa"));
+    DrillSidewaysResult r = new DrillSideways(searcher, taxoReader).search(null, ddq, 10, fsp);
+
+    assertEquals(1, 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: 2010=1", toString(r.facetResults.get(0)));
+    // Author is drill-sideways + drill-down: Lisa
+    // (drill-down) published once, and Bob
+    // published once:
+    assertEquals("Author: Lisa=1 Bob=1", toString(r.facetResults.get(1)));
+
+    searcher.getIndexReader().close();
+    taxoReader.close();
+    dir.close();
+    taxoDir.close();
+  }
+
+  private static class Doc implements Comparable<Doc> {
+    String id;
+    String contentToken;
+
+    // -1 if the doc is missing this dim, else the index
+    // -into the values for this dim:
+    int[] dims;
+
+    // 2nd value per dim for the doc (so we test
+    // multi-valued fields):
+    int[] dims2;
+    boolean deleted;
+
+    @Override
+    public int compareTo(Doc other) {
+      return id.compareTo(other.id);
+    }
+  }
+
+  private double aChance, bChance, cChance;
+
+  private String randomContentToken(boolean isQuery) {
+    double d = random().nextDouble();
+    if (isQuery) {
+      if (d < 0.33) {
+        return "a";
+      } else if (d < 0.66) {
+        return "b";
+      } else {
+        return "c";
+      }
+    } else {
+      if (d <= aChance) {
+        return "a";
+      } else if (d < aChance + bChance) {
+        return "b";
+      } else {
+        return "c";
+      }
+    }
+  }
+
+  public void testRandom() throws Exception {
+
+    while (aChance == 0.0) {
+      aChance = random().nextDouble();
+    }
+    while (bChance == 0.0) {
+      bChance = random().nextDouble();
+    }
+    while (cChance == 0.0) {
+      cChance = random().nextDouble();
+    }
+    /*
+    aChance = .01;
+    bChance = 0.5;
+    cChance = 1.0;
+    */
+    double sum = aChance + bChance + cChance;
+    aChance /= sum;
+    bChance /= sum;
+    cChance /= sum;
+
+    int numDims = _TestUtil.nextInt(random(), 2, 5);
+    //int numDims = 3;
+    int numDocs = atLeast(3000);
+    //int numDocs = 20;
+    if (VERBOSE) {
+      System.out.println("numDims=" + numDims + " numDocs=" + numDocs + " aChance=" + aChance + " bChance=" + bChance + " cChance=" + cChance);
+    }
+    String[][] dimValues = new String[numDims][];
+    int valueCount = 2;
+    for(int dim=0;dim<numDims;dim++) {
+      Set<String> values = new HashSet<String>();
+      while (values.size() < valueCount) {
+        String s = _TestUtil.randomRealisticUnicodeString(random());
+        //String s = _TestUtil.randomSimpleString(random());
+        if (s.length() > 0) {
+          values.add(s);
+        }
+      } 
+      dimValues[dim] = values.toArray(new String[values.size()]);
+      valueCount *= 2;
+    }
+
+    List<Doc> docs = new ArrayList<Doc>();
+    for(int i=0;i<numDocs;i++) {
+      Doc doc = new Doc();
+      doc.id = ""+i;
+      doc.contentToken = randomContentToken(false);
+      doc.dims = new int[numDims];
+      doc.dims2 = new int[numDims];
+      for(int dim=0;dim<numDims;dim++) {
+        if (random().nextInt(5) == 3) {
+          // This doc is missing this dim:
+          doc.dims[dim] = -1;
+        } else if (dimValues[dim].length <= 4) {
+          int dimUpto = 0;
+          doc.dims[dim] = dimValues[dim].length-1;
+          while (dimUpto < dimValues[dim].length) {
+            if (random().nextBoolean()) {
+              doc.dims[dim] = dimUpto;
+              break;
+            }
+            dimUpto++;
+          }
+        } else {
+          doc.dims[dim] = random().nextInt(dimValues[dim].length);
+        }
+
+        if (random().nextInt(5) == 3) {
+          // 2nd value:
+          doc.dims2[dim] = random().nextInt(dimValues[dim].length);
+        } else {
+          doc.dims2[dim] = -1;
+        }
+      }
+      docs.add(doc);
+    }
+
+    Directory d = newDirectory();
+    Directory td = newDirectory();
+
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    iwc.setInfoStream(InfoStream.NO_OUTPUT);
+    RandomIndexWriter w = new RandomIndexWriter(random(), d, iwc);
+    DirectoryTaxonomyWriter tw = new DirectoryTaxonomyWriter(td, IndexWriterConfig.OpenMode.CREATE);
+    facetFields = new FacetFields(tw);
+
+    for(Doc rawDoc : docs) {
+      Document doc = new Document();
+      doc.add(newStringField("id", rawDoc.id, Field.Store.YES));
+      doc.add(newStringField("content", rawDoc.contentToken, Field.Store.NO));
+      List<CategoryPath> paths = new ArrayList<CategoryPath>();
+
+      if (VERBOSE) {
+        System.out.println("  doc id=" + rawDoc.id + " token=" + rawDoc.contentToken);
+      }
+      for(int dim=0;dim<numDims;dim++) {
+        int dimValue = rawDoc.dims[dim];
+        if (dimValue != -1) {
+          paths.add(new CategoryPath("dim" + dim, dimValues[dim][dimValue]));
+          doc.add(new StringField("dim" + dim, dimValues[dim][dimValue], Field.Store.YES));
+          if (VERBOSE) {
+            System.out.println("    dim" + dim + "=" + dimValues[dim][dimValue]);
+          }
+        }
+        int dimValue2 = rawDoc.dims2[dim];
+        if (dimValue2 != -1) {
+          paths.add(new CategoryPath("dim" + dim, dimValues[dim][dimValue2]));
+          doc.add(new StringField("dim" + dim, dimValues[dim][dimValue2], Field.Store.YES));
+          if (VERBOSE) {
+            System.out.println("      dim" + dim + "=" + dimValues[dim][dimValue2]);
+          }
+        }
+      }
+      if (!paths.isEmpty()) {
+        facetFields.addFields(doc, paths);
+      }
+      w.addDocument(doc);
+    }
+
+    if (random().nextBoolean()) {
+      // Randomly delete a few docs:
+      int numDel = _TestUtil.nextInt(random(), 1, (int) (numDocs*0.05));
+      if (VERBOSE) {
+        System.out.println("delete " + numDel);
+      }
+      int delCount = 0;
+      while (delCount < numDel) {
+        Doc doc = docs.get(random().nextInt(docs.size()));
+        if (!doc.deleted) {
+          if (VERBOSE) {
+            System.out.println("  delete id=" + doc.id);
+          }
+          doc.deleted = true;
+          w.deleteDocuments(new Term("id", doc.id));
+          delCount++;
+        }
+      }
+    }
+
+    if (random().nextBoolean()) {
+      if (VERBOSE) {
+        System.out.println("TEST: forceMerge(1)...");
+      }
+      w.forceMerge(1);
+    }
+    IndexReader r = w.getReader();
+    w.close();
+    if (VERBOSE) {
+      System.out.println("r.numDocs() = " + r.numDocs());
+    }
+
+    // NRT open
+    TaxonomyReader tr = new DirectoryTaxonomyReader(tw);
+    tw.close();
+
+    List<FacetRequest> requests = new ArrayList<FacetRequest>();
+    for(int i=0;i<numDims;i++) {
+      requests.add(new CountFacetRequest(new CategoryPath("dim" + i), dimValues[numDims-1].length));
+    }
+
+    FacetSearchParams fsp = new FacetSearchParams(requests);
+    IndexSearcher s = new IndexSearcher(r);
+
+    int numIters = atLeast(10);
+
+    for(int iter=0;iter<numIters;iter++) {
+      String contentToken = random().nextInt(30) == 17 ? null : randomContentToken(true);
+      int numDrillDown = _TestUtil.nextInt(random(), 1, Math.min(4, numDims));
+      String[][] drillDowns = new String[numDims][];
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter + " baseQuery=" + contentToken + " numDrillDown=" + numDrillDown);
+      }
+      int count = 0;
+      while (count < numDrillDown) {
+        int dim = random().nextInt(numDims);
+        if (drillDowns[dim] == null) {
+          if (random().nextBoolean()) {
+            // Drill down on one value:
+            drillDowns[dim] = new String[] {dimValues[dim][random().nextInt(dimValues[dim].length)]};
+          } else {
+            int orCount = _TestUtil.nextInt(random(), 1, Math.min(5, dimValues[dim].length));
+            drillDowns[dim] = new String[orCount];
+            for(int i=0;i<orCount;i++) {
+              while (true) {
+                String value = dimValues[dim][random().nextInt(dimValues[dim].length)];
+                for(int j=0;j<i;j++) {
+                  if (value.equals(drillDowns[dim][j])) {
+                    value = null;
+                    break;
+                  }
+                }
+                if (value != null) {
+                  drillDowns[dim][i] = value;
+                  break;
+                }
+              }
+            }
+          }
+          if (VERBOSE) {
+            System.out.println("  dim" + dim + "=" + Arrays.toString(drillDowns[dim]));
+          }
+          count++;
+        }
+      }
+
+      Query baseQuery;
+      if (contentToken == null) {
+        baseQuery = new MatchAllDocsQuery();
+      } else {
+        baseQuery = new TermQuery(new Term("content", contentToken));
+      }
+
+      DrillDownQuery ddq = new DrillDownQuery(fsp.indexingParams, baseQuery);
+
+      for(int dim=0;dim<numDims;dim++) {
+        if (drillDowns[dim] != null) {
+          CategoryPath[] paths = new CategoryPath[drillDowns[dim].length];
+          int upto = 0;
+          for(String value : drillDowns[dim]) {
+            paths[upto++] = new CategoryPath("dim" + dim, value);
+          }
+          ddq.add(paths);
+        }
+      }
+
+      Filter filter;
+      if (random().nextInt(7) == 6) {
+        if (VERBOSE) {
+          System.out.println("  only-even filter");
+        }
+        filter = new Filter() {
+            @Override
+            public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
+              int maxDoc = context.reader().maxDoc();
+              final FixedBitSet bits = new FixedBitSet(maxDoc);
+              for(int docID=0;docID < maxDoc;docID++) {
+                // Keeps only the even ids:
+                if ((acceptDocs == null || acceptDocs.get(docID)) && ((Integer.parseInt(context.reader().document(docID).get("id")) & 1) == 0)) {
+                  bits.set(docID);
+                }
+              }
+              return bits;
+            }
+          };
+      } else {
+        filter = null;
+      }
+
+      // Verify docs are always collected in order:
+      new DrillSideways(s, tr).search(ddq,
+                           new Collector() {
+                             int lastDocID;
+
+                             @Override
+                             public void setScorer(Scorer s) {
+                             }
+
+                             @Override
+                             public void collect(int doc) {
+                               assert doc > lastDocID;
+                               lastDocID = doc;
+                             }
+
+                             @Override
+                             public void setNextReader(AtomicReaderContext context) {
+                               lastDocID = -1;
+                             }
+
+                             @Override
+                             public boolean acceptsDocsOutOfOrder() {
+                               return false;
+                             }
+                           }, fsp);
+
+      SimpleFacetResult expected = slowDrillSidewaysSearch(s, docs, contentToken, drillDowns, dimValues, filter);
+
+      Sort sort = new Sort(new SortField("id", SortField.Type.STRING));
+      DrillSidewaysResult actual = new DrillSideways(s, tr).search(ddq, filter, null, numDocs, sort, true, true, fsp);
+
+      TopDocs hits = s.search(baseQuery, numDocs);
+      Map<String,Float> scores = new HashMap<String,Float>();
+      for(ScoreDoc sd : hits.scoreDocs) {
+        scores.put(s.doc(sd.doc).get("id"), sd.score);
+      }
+      
+      verifyEquals(dimValues, s, expected, actual, scores);
+    }
+
+    tr.close();
+    r.close();
+    td.close();
+    d.close();
+  }
+
+  private static class Counters {
+    int[][] counts;
+
+    public Counters(String[][] dimValues) {
+      counts = new int[dimValues.length][];
+      for(int dim=0;dim<dimValues.length;dim++) {
+        counts[dim] = new int[dimValues[dim].length];
+      }
+    }
+
+    public void inc(int[] dims, int[] dims2) {
+      inc(dims, dims2, -1);
+    }
+
+    public void inc(int[] dims, int[] dims2, int onlyDim) {
+      assert dims.length == counts.length;
+      assert dims2.length == counts.length;
+      for(int dim=0;dim<dims.length;dim++) {
+        if (onlyDim == -1 || dim == onlyDim) {
+          if (dims[dim] != -1) {
+            counts[dim][dims[dim]]++;
+          }
+          if (dims2[dim] != -1 && dims2[dim] != dims[dim]) {
+            counts[dim][dims2[dim]]++;
+          }
+        }
+      }
+    }
+  }
+
+  private static class SimpleFacetResult {
+    List<Doc> hits;
+    int[][] counts;
+  }
+
+  private SimpleFacetResult slowDrillSidewaysSearch(IndexSearcher s, List<Doc> docs, String contentToken, String[][] drillDowns,
+                                                    String[][] dimValues, Filter onlyEven) throws Exception {
+    int numDims = dimValues.length;
+
+    List<Doc> hits = new ArrayList<Doc>();
+    Counters drillDownCounts = new Counters(dimValues);
+    Counters[] drillSidewaysCounts = new Counters[dimValues.length];
+    for(int dim=0;dim<numDims;dim++) {
+      drillSidewaysCounts[dim] = new Counters(dimValues);
+    }
+
+    if (VERBOSE) {
+      System.out.println("  compute expected");
+    }
+
+    nextDoc: for(Doc doc : docs) {
+      if (doc.deleted) {
+        continue;
+      }
+      if (onlyEven != null & (Integer.parseInt(doc.id) & 1) != 0) {
+        continue;
+      }
+      if (contentToken == null || doc.contentToken.equals(contentToken)) {
+        int failDim = -1;
+        for(int dim=0;dim<numDims;dim++) {
+          if (drillDowns[dim] != null) {
+            String docValue = doc.dims[dim] == -1 ? null : dimValues[dim][doc.dims[dim]];
+            String docValue2 = doc.dims2[dim] == -1 ? null : dimValues[dim][doc.dims2[dim]];
+            boolean matches = false;
+            for(String value : drillDowns[dim]) {
+              if (value.equals(docValue) || value.equals(docValue2)) {
+                matches = true;
+                break;
+              }
+            }
+            if (!matches) {
+              if (failDim == -1) {
+                // Doc could be a near-miss, if no other dim fails
+                failDim = dim;
+              } else {
+                // Doc isn't a hit nor a near-miss
+                continue nextDoc;
+              }
+            }
+          }
+        }
+
+        if (failDim == -1) {
+          if (VERBOSE) {
+            System.out.println("    exp: id=" + doc.id + " is a hit");
+          }
+          // Hit:
+          hits.add(doc);
+          drillDownCounts.inc(doc.dims, doc.dims2);
+          for(int dim=0;dim<dimValues.length;dim++) {
+            drillSidewaysCounts[dim].inc(doc.dims, doc.dims2);
+          }
+        } else {
+          if (VERBOSE) {
+            System.out.println("    exp: id=" + doc.id + " is a near-miss on dim=" + failDim);
+          }
+          drillSidewaysCounts[failDim].inc(doc.dims, doc.dims2, failDim);
+        }
+      }
+    }
+
+    Map<String,Integer> idToDocID = new HashMap<String,Integer>();
+    for(int i=0;i<s.getIndexReader().maxDoc();i++) {
+      idToDocID.put(s.doc(i).get("id"), i);
+    }
+
+    Collections.sort(hits);
+
+    SimpleFacetResult res = new SimpleFacetResult();
+    res.hits = hits;
+    res.counts = new int[numDims][];
+    for(int dim=0;dim<numDims;dim++) {
+      if (drillDowns[dim] != null) {
+        res.counts[dim] = drillSidewaysCounts[dim].counts[dim];
+      } else {
+        res.counts[dim] = drillDownCounts.counts[dim];
+      }
+    }
+
+    return res;
+  }
+
+  void verifyEquals(String[][] dimValues, IndexSearcher s, SimpleFacetResult expected, DrillSidewaysResult actual, Map<String,Float> scores) throws Exception {
+    if (VERBOSE) {
+      System.out.println("  verify totHits=" + expected.hits.size());
+    }
+    assertEquals(expected.hits.size(), actual.hits.totalHits);
+    assertEquals(expected.hits.size(), actual.hits.scoreDocs.length);
+    for(int i=0;i<expected.hits.size();i++) {
+      if (VERBOSE) {
+        System.out.println("    hit " + i + " expected=" + expected.hits.get(i).id);
+      }
+      assertEquals(expected.hits.get(i).id,
+                   s.doc(actual.hits.scoreDocs[i].doc).get("id"));
+      // Score should be IDENTICAL:
+      assertEquals(scores.get(expected.hits.get(i).id), actual.hits.scoreDocs[i].score, 0.0f);
+    }
+    assertEquals(expected.counts.length, actual.facetResults.size());
+    for(int dim=0;dim<expected.counts.length;dim++) {
+      if (VERBOSE) {
+        System.out.println("    dim" + dim);
+        System.out.println("      actual");
+      }
+      FacetResult fr = actual.facetResults.get(dim);
+      Map<String,Integer> actualValues = new HashMap<String,Integer>();
+      for(FacetResultNode childNode : fr.getFacetResultNode().subResults) {
+        actualValues.put(childNode.label.components[1], (int) childNode.value);
+        if (VERBOSE) {
+          System.out.println("        " + childNode.label.components[1] + ": " + (int) childNode.value);
+        }
+      }
+
+      if (VERBOSE) {
+        System.out.println("      expected");
+      }
+
+      int setCount = 0;
+      for(int i=0;i<dimValues[dim].length;i++) {
+        String value = dimValues[dim][i];
+        if (expected.counts[dim][i] != 0) {
+          if (VERBOSE) {
+            System.out.println("        " + value + ": " + expected.counts[dim][i]);
+          } 
+          assertTrue(actualValues.containsKey(value));
+          assertEquals(expected.counts[dim][i], actualValues.get(value).intValue());
+          setCount++;
+        } else {
+          assertFalse(actualValues.containsKey(value));
+        }
+      }
+
+      assertEquals(setCount, actualValues.size());
+    }
+  }
+
+  /** Just gathers counts of values under the dim. */
+  private String toString(FacetResult fr) {
+    StringBuilder b = new StringBuilder();
+    FacetResultNode node = fr.getFacetResultNode();
+    b.append(node.label);
+    b.append(":");
+    for(FacetResultNode childNode : node.subResults) {
+      b.append(' ');
+      b.append(childNode.label.components[1]);
+      b.append('=');
+      b.append((int) childNode.value);
+    }
+    return b.toString();
+  }
+}
+

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKInEachNodeResultHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKInEachNodeResultHandler.java?rev=1449972&r1=1449971&r2=1449972&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKInEachNodeResultHandler.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKInEachNodeResultHandler.java Tue Feb 26 00:02:16 2013
@@ -259,18 +259,30 @@ public class TestTopKInEachNodeResultHan
         assertEquals(0, node.subResults.size());
       }
       
-      fr = facetResults.get(6); // a/b, depth=0, K=2
+      fr = facetResults.get(6); // Doctor, depth=0, K=2
       hasDoctor |= "Doctor".equals(fr.getFacetRequest().categoryPath.components[0]);
       assertEquals(0, fr.getNumValidDescendants()); // 0 descendants but rootnode
       parentRes = fr.getFacetResultNode();
-      assertEquals(8.0, parentRes.value, Double.MIN_VALUE);
+      assertEquals(0.0, parentRes.value, Double.MIN_VALUE);
       assertEquals(0, parentRes.subResults.size());
       hasDoctor |= "Doctor".equals(fr.getFacetRequest().categoryPath.components[0]);
 
       // doctor, depth=1, K=2
-      assertFalse("Shouldn't have found anything for a FacetRequest " +
-          "of a facet that doesn't exist in the index.", hasDoctor);
-      assertEquals("Shouldn't have found more than seven request.", 7, facetResults.size());
+      assertTrue("Should have found an empty FacetResult " +
+          "for a facet that doesn't exist in the index.", hasDoctor);
+      assertEquals("Shouldn't have found more than 8 request.", 8, facetResults.size());
+
+      fr = facetResults.get(7); // a/b, depth=0, K=2
+      assertEquals(0, fr.getNumValidDescendants());
+      parentRes = fr.getFacetResultNode();
+      assertEquals(8.0, parentRes.value, Double.MIN_VALUE);
+      assertEquals(0, parentRes.subResults.size());
+      i = 0;
+      for (FacetResultNode node : parentRes.subResults) {
+        assertEquals(expectedValues3[i++], node.value, Double.MIN_VALUE);
+        assertEquals(0, node.subResults.size());
+      }
+
       ir.close();
       tr.close();
       iDir.close();

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKResultsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKResultsHandler.java?rev=1449972&r1=1449971&r2=1449972&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKResultsHandler.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKResultsHandler.java Tue Feb 26 00:02:16 2013
@@ -203,8 +203,8 @@ public class TestTopKResultsHandler exte
       List<FacetResult> facetResults = fc.getFacetResults();
       
       assertEquals("Shouldn't have found anything for a FacetRequest "
-          + "of a facet that doesn't exist in the index.", 0, facetResults.size());
-
+          + "of a facet that doesn't exist in the index.", 1, facetResults.size());
+      assertEquals("Miau Hattulla", facetResults.get(0).getFacetResultNode().label.components[0]);
       closeAll();
     }
   }

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/AssertingIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/AssertingIndexSearcher.java?rev=1449972&r1=1449971&r2=1449972&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/AssertingIndexSearcher.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/AssertingIndexSearcher.java Tue Feb 26 00:02:16 2013
@@ -98,6 +98,9 @@ public class AssertingIndexSearcher exte
 
       @Override
       public boolean scoresDocsOutOfOrder() {
+        // TODO: if this returns false, we should wrap
+        // Scorer with AssertingScorer that confirms docIDs
+        // are in order?
         return w.scoresDocsOutOfOrder();
       }
     };