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 2014/01/05 22:05:03 UTC

svn commit: r1555629 [5/7] - in /lucene/dev/branches/lucene5376: ./ dev-tools/ dev-tools/maven/solr/contrib/map-reduce/ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/common/src/java/org/apache/lucene/analysis/ckb/ lucene/analysis/com...

Modified: lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/SearchHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/SearchHandler.java?rev=1555629&r1=1555628&r2=1555629&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/SearchHandler.java (original)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/SearchHandler.java Sun Jan  5 21:04:58 2014
@@ -33,23 +33,18 @@ import java.util.Set;
 import java.util.TreeMap;
 
 import org.apache.lucene.document.FieldType.NumericType;
-import org.apache.lucene.facet.params.FacetIndexingParams;
-import org.apache.lucene.facet.params.FacetSearchParams;
+import org.apache.lucene.facet.DrillDownQuery;
+import org.apache.lucene.facet.DrillSideways;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.LabelAndValue;
 import org.apache.lucene.facet.range.LongRange;
+import org.apache.lucene.facet.range.LongRangeFacetCounts;
 import org.apache.lucene.facet.range.Range;
-import org.apache.lucene.facet.range.RangeFacetRequest;
-import org.apache.lucene.facet.search.CachedOrdsCountingFacetsAggregator;
-import org.apache.lucene.facet.search.CountFacetRequest;
-import org.apache.lucene.facet.search.DrillDownQuery;
-import org.apache.lucene.facet.search.DrillSideways;
-import org.apache.lucene.facet.search.FacetArrays;
-import org.apache.lucene.facet.search.FacetRequest;
-import org.apache.lucene.facet.search.FacetResult;
-import org.apache.lucene.facet.search.FacetResultNode;
-import org.apache.lucene.facet.search.FacetsAccumulator;
-import org.apache.lucene.facet.search.FacetsAggregator;
-import org.apache.lucene.facet.search.SearcherTaxonomyManager.SearcherAndTaxonomy;
-import org.apache.lucene.facet.taxonomy.CategoryPath;
+import org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts;
+import org.apache.lucene.facet.taxonomy.SearcherTaxonomyManager.SearcherAndTaxonomy;
+import org.apache.lucene.facet.taxonomy.TaxonomyFacetCounts;
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.FieldInfo.DocValuesType;
@@ -110,11 +105,9 @@ import org.apache.lucene.server.FieldDef
 import org.apache.lucene.server.FinishRequest;
 import org.apache.lucene.server.GlobalState;
 import org.apache.lucene.server.IndexState;
-import org.apache.lucene.server.MultiFacetsAccumulator;
 import org.apache.lucene.server.MyIndexSearcher;
 import org.apache.lucene.server.RecencyBlendedFieldComparatorSource;
 import org.apache.lucene.server.SVJSONPassageFormatter;
-import org.apache.lucene.server.TopFacetsCache;
 import org.apache.lucene.server.WholeMVJSONPassageFormatter;
 import org.apache.lucene.server.params.*;
 import org.apache.lucene.server.params.PolyType.PolyEntry;
@@ -126,6 +119,8 @@ import net.minidev.json.JSONObject;
 import net.minidev.json.JSONValue;
 import net.minidev.json.parser.ParseException;
 
+// nocommit why no double range faceting?
+
 /** Handles {@code search}. */
 public class SearchHandler extends Handler {
 
@@ -251,6 +246,11 @@ public class SearchHandler extends Handl
     QUERY_TYPE_WRAP.set(QUERY_TYPE);
   }
 
+  public static final StructType SEARCHER_VERSION_TYPE = new StructType(
+                new Param("indexGen", "Search a generation previously returned by an indexing operation such as #addDocument.  Use this to search a non-committed (near-real-time) view of the index.", new LongType()),
+                new Param("snapshot", "Search a snapshot previously created with #createSnapshot", new StringType()),
+                new Param("version", "Search a specific searcher version.  This is typically used by follow-on searches (e.g., user clicks next page, drills down, or changes sort, etc.) to get the same searcher used by the original search.", new LongType()));
+
   final static Type LOCALE_TYPE = new StructType(new Param("language", "Locale language", new StringType()),
                                                  new Param("country", "Locale country", new StringType()),
                                                  new Param("variant", "Locale variant", new StringType()));
@@ -280,7 +280,7 @@ public class SearchHandler extends Handl
   private final static StructType TYPE =
     new StructType(
         new Param("indexName", "Which index to search", new StringType()),
-        new Param("timeStamp", "The 'current' timestamp to use, e.g. for blended sorting; typically this should be fixed and reused from the returned value from the first query in a session", new LongType()),
+        new Param("timeStamp", "The 'current' timestamp to use, e.g. for blended sorting; typically this should be fixed and reused from the returned value from the first query in a session so that as the user does follow-on paging, drill-downs, etc., the results don't unexpectedly changing.", new LongType()),
         new Param("queryText", "Query text to parse using the specified QueryParser.", new StringType()),
         new Param("filter", "Filter to apply to search", FILTER_TYPE),
         new Param("queryParser", "Which QueryParser to use; by default MultiFieldQUeryParser searching all indexed fields will be used", QUERY_PARSER_TYPE),
@@ -320,10 +320,7 @@ public class SearchHandler extends Handl
                                  new Param("doTotalGroupCount", "If true, return the total number of groups (at possibly highish added CPU cost)",
                                            new BooleanType(), false))),
         new Param("searcher", "Specific searcher version to use for searching.  There are three different ways to specify a searcher version.",
-            new StructType(
-                new Param("indexGen", "Search a generation previously returned by an indexing operation such as #addDocument.  Use this to search a non-committed (near-real-time) view of the index.", new LongType()),
-                new Param("snapshot", "Search a snapshot previously created with #createSnapshot", new StringType()),
-                new Param("version", "Search a specific searcher version.  This is typically used by follow-on searches (e.g., user clicks next page, drills down, or changes sort, etc.) to get the same searcher used by the original search.", new LongType()))),
+                  SEARCHER_VERSION_TYPE),
         new Param("startHit", "Which hit to start from (for pagination).", new IntType(), 0),
         new Param("topHits", "How many top hits to retrieve.", new IntType(), 10),
         new Param("searchAfter", "Only return hits after the specified hit; this is useful for deep paging",
@@ -360,6 +357,8 @@ public class SearchHandler extends Handl
         new Param("facets", "Which facets to retrieve.",
                   new ListType(
                                new StructType(
+                                   // nocommit rename to field:
+                                   new Param("dim", "Dimension (field).", new StringType()),
                                    new Param("path", "Prefix path to facet 'under'.",
                                              new OrType(new StringType(), new ListType(new StringType()))),
                                    new Param("numericRanges", "Custom numeric ranges.  Field must be indexed with facet=numericRange.",
@@ -371,13 +370,13 @@ public class SearchHandler extends Handl
                                                           new Param("maxInclusive", "True if the max value is inclusive", new BooleanType())))),
                                    new Param("autoDrillDown", "True if single-child facet should be auto-expanded (not yet implemented!).", new BooleanType()),
                                    new Param("useOrdsCache", "True if the ordinals cache should be used.", new BooleanType(), false),
-                                   new Param("topN", "How many top facets to keep.", new IntType(), 10)))),
+                                   new Param("topN", "How many top facets to return.", new IntType(), 7)))),
         new Param("drillDowns", "Facet drill down filters to apply.",
                   new ListType(new StructType(
                                    new Param("field", "Field name to drill down on.", new StringType()),
-                                   new Param("values", "Which (OR'd) values to allow.  If an element a String, it's a flat facet value; if it's a list of String then it's a hierarchy path.",
-                                             new ListType(new OrType(new StringType(),
-                                                                     new ListType(new StringType()))))))),
+                                   new Param("query", "Sub-query for drill down (e.g., use NumericRangeQuery to drill down on dynamic ranges).", QUERY_TYPE),
+                                   new Param("value", "Which values to allow, either String or array of String.  Drill down on the same field more than once to OR multiple drill-down values for that field.",
+                                             new OrType(new StringType(), new ListType(new StringType())))))),
         new Param("sort", "Sort hits by field (default is by relevance).",
                   new StructType(
                       new Param("doMaxScore", "Compute the max score across all hits (costs added CPU).", new BooleanType(), false),
@@ -630,7 +629,7 @@ public class SearchHandler extends Handl
     return config;
   }
 
-  private Sort parseSort(long timeStamp, IndexState state, List<Object> fields) {
+  private static Sort parseSort(long timeStamp, IndexState state, List<Object> fields) {
     List<SortField> sortFields = new ArrayList<SortField>();
     for(Object _sub : fields) {
       Request sub = (Request) _sub;
@@ -802,7 +801,7 @@ public class SearchHandler extends Handl
     }
   }
 
-  private Filter parseFilter(long timeStamp, Request topRequest, IndexState state, Request r) {
+  private static Filter parseFilter(long timeStamp, Request topRequest, IndexState state, Request r) {
     Filter f;
 
     Request.PolyResult pr = r.getPoly("class");
@@ -887,7 +886,8 @@ public class SearchHandler extends Handl
   }
 
   @SuppressWarnings("unchecked")
-  private Query parseQuery(long timeStamp, Request topRequest, IndexState state, Request r, String field, Map<ToParentBlockJoinQuery,BlockJoinQueryChild> useBlockJoinCollector) {
+  private static Query parseQuery(long timeStamp, Request topRequest, IndexState state, Request r, String field,
+                                  Map<ToParentBlockJoinQuery,BlockJoinQueryChild> useBlockJoinCollector) {
     Query q;
     Request.PolyResult pr = r.getPoly("class");
     if (r.hasParam("field")) {
@@ -1176,7 +1176,7 @@ public class SearchHandler extends Handl
 
   /** If field is non-null it overrides any specified
    *  defaultField. */
-  private QueryParser createQueryParser(IndexState state, Request r, String field) {
+  private static QueryParser createQueryParser(IndexState state, Request r, String field) {
 
     if (r.hasParam("queryParser")) {
       r = r.getStruct("queryParser");
@@ -1272,10 +1272,51 @@ public class SearchHandler extends Handl
     public BreakIterator breakIterator;
   }
 
+  public static SearcherAndTaxonomy getSearcherAndTaxonomy(Request request, IndexState state, JSONObject diagnostics) throws InterruptedException, IOException {
+    // Figure out which searcher to use:
+    final long searcherVersion;
+    final IndexState.Gens searcherSnapshot;
+    if (request.hasParam("searcher")) {
+      Request r2 = request.getStruct("searcher");
+      if (r2.hasParam("indexGen")) {
+        long indexGen = r2.getLong("indexGen");
+        long t0 = System.nanoTime();
+        state.reopenThread.waitForGeneration(indexGen);
+        if (diagnostics != null) {
+          diagnostics.put("nrtWaitMS", (System.nanoTime() - t0)/1000000);
+        }
+        searcherVersion = -1;
+        searcherSnapshot = null;
+      } else if (r2.hasParam("version")) {
+        searcherVersion = r2.getLong("version");
+        searcherSnapshot = null;
+      } else if (r2.hasParam("snapshot")) {
+        searcherSnapshot = new IndexState.Gens(r2, "snapshot");
+        Long v = state.snapshotGenToVersion.get(searcherSnapshot.indexGen);
+        if (v == null) {
+          r2.fail("snapshot", "unrecognized snapshot \"" + searcherSnapshot.id + "\"");
+        }
+        searcherVersion = v.longValue();
+      } else {
+        request.fail("searcher", "must specify exactly one of indexGen, version or snapshot");
+        // Dead code but compiler disagrees:
+        searcherSnapshot = null;
+        searcherVersion = -1;
+      }
+    } else {
+      searcherSnapshot = null;
+      searcherVersion = -1;
+    }
+
+    // nocommit merge this method into here; don't need
+    // separate method w/ intermediate vars
+    return getSearcherAndTaxonomy(request, state, searcherVersion, searcherSnapshot, diagnostics);
+  }
+
   /** Retrieve the {@link SearcherAndTaxonomy} by version or
    *  snapshot. */
-  public static SearcherAndTaxonomy getSearcherAndTaxonomy(Request request, IndexState state, long version,
-                                                           IndexState.Gens snapshot, JSONObject diagnostics) throws IOException {
+  private static SearcherAndTaxonomy getSearcherAndTaxonomy(Request request, IndexState state, long version,
+                                                            IndexState.Gens snapshot, JSONObject diagnostics) throws IOException {
     SearcherAndTaxonomy s;
 
     if (version == -1) {
@@ -1340,27 +1381,51 @@ public class SearchHandler extends Handl
     return s;
   }
 
-  @SuppressWarnings("unchecked")
-  @Override
-  public FinishRequest handle(final IndexState state, final Request r, Map<String,List<String>> params) throws Exception {
+  /** Fold in any drillDowns requests into the query. */
+  private static DrillDownQuery addDrillDowns(long timeStamp, IndexState state, Request r, Query q) {
+    // Always create a DrillDownQuery; if there
+    // are no drill-downs it will just rewrite to the
+    // original query:
+    DrillDownQuery ddq = new DrillDownQuery(state.facetsConfig, q);
 
-    state.verifyStarted(r);
+    if (r.hasParam("drillDowns")) {
+      List<Object> drillDownList = r.getList("drillDowns");
+      if (!drillDownList.isEmpty()) {
+        for(Object o : drillDownList) {
+          Request fr = (Request) o;
+          FieldDef fd = state.getField(fr, "field");
 
-    final Map<ToParentBlockJoinQuery,BlockJoinQueryChild> useBlockJoinCollector = new HashMap<ToParentBlockJoinQuery,BlockJoinQueryChild>();
+          if (fr.hasParam("query")) {
+            // Drill down by query:
+            ddq.add(fd.name, parseQuery(timeStamp, null, state, r.getStruct("query"), fd.name, null));
+          } else {
+            String[] path;
+            if (fr.isString("value")) {
+              path = new String[] {fr.getString("value")};
+            } else {
+              List<Object> values = fr.getList("value");
+              path = new String[values.size()];
+              for(int i=0;i<path.length;i++) {
+                path[i] = (String) values.get(i);
+              }
+            }
 
-    final long timeStamp;
-    if (r.hasParam("timeStamp")) {
-      timeStamp = r.getLong("timeStamp");
-    } else {
-      timeStamp = System.currentTimeMillis()/1000;
+            ddq.add(fd.name, path);
+          }
+        }
+      }
     }
 
-    String queryText;
-    final Query q;
+    return ddq;
+  }
+
+  private static Query extractQuery(IndexState state, Request r, long timeStamp,
+                                    Map<ToParentBlockJoinQuery,BlockJoinQueryChild> useBlockJoinCollector) throws Exception {
+    Query q;
     if (r.hasParam("queryText")) {
       QueryParser queryParser = createQueryParser(state, r, null);
 
-      queryText = r.getString("queryText");
+      String queryText = r.getString("queryText");
 
       if (queryText != null) {
         try {
@@ -1379,146 +1444,191 @@ public class SearchHandler extends Handl
       q = new MatchAllDocsQuery();
     }
 
-    final JSONObject diagnostics = new JSONObject();
+    return q;
+  }
 
-    // Figure out which searcher to use:
-    final long searcherVersion;
-    final IndexState.Gens searcherSnapshot;
-    if (r.hasParam("searcher")) {
-      Request r2 = r.getStruct("searcher");
-      if (r2.hasParam("indexGen")) {
-        long indexGen = r2.getLong("indexGen");
-        long t0 = System.nanoTime();
-        state.reopenThread.waitForGeneration(indexGen);
-        diagnostics.put("nrtWaitMS", (System.nanoTime() - t0)/1000000);
-        searcherVersion = -1;
-        searcherSnapshot = null;
-      } else if (r2.hasParam("version")) {
-        searcherVersion = r2.getLong("version");
-        searcherSnapshot = null;
-      } else if (r2.hasParam("snapshot")) {
-        searcherSnapshot = new IndexState.Gens(r2, "snapshot");
-        Long v = state.snapshotGenToVersion.get(searcherSnapshot.indexGen);
-        if (v == null) {
-          r2.fail("snapshot", "unrecognized snapshot \"" + searcherSnapshot.id + "\"");
-        }
-        searcherVersion = v.longValue();
-      } else {
-        r.fail("searcher", "must specify exactly one of indexGen, version or snapshot");
-        // Dead code but compiler disagrees:
-        searcherSnapshot = null;
-        searcherVersion = -1;
-      }
-    } else {
-      searcherSnapshot = null;
-      searcherVersion = -1;
+  private static JSONObject buildFacetResultJSON(FacetResult result) {
+    JSONObject o = new JSONObject();
+    JSONArray counts = new JSONArray();
+    o.put("counts", counts);
+    o.put("childCount", result.childCount);
+    JSONArray pair = new JSONArray();
+    counts.add(pair);
+    pair.add("top");
+    pair.add(result.value);
+    for(LabelAndValue labelValue : result.labelValues) {
+      pair = new JSONArray();
+      counts.add(pair);
+      pair.add(labelValue.label);
+      pair.add(labelValue.value);
     }
 
-    final int topHits = r.getInt("topHits");
-    final int startHit = r.getInt("startHit");
+    return o;
+  }
 
-    final Sort sort;
-    final boolean doMaxScore;
-    final boolean doDocScores;
-
-    if (r.hasParam("sort")) {
-      // Sort by fields:
-      Request sr = r.getStruct("sort");
-
-      sort = parseSort(timeStamp, state, sr.getList("fields"));
-
-      //System.out.println("sort=" + sort);
-      doMaxScore = sr.getBoolean("doMaxScore");
-      doDocScores = sr.getBoolean("doDocScores");
-    } else {
-      sort = null;
-      doMaxScore = false;
-      doDocScores = false;
-    }
+  static void fillFacetResults(Request r, SearcherAndTaxonomy s, FacetsCollector drillDowns,
+                               FacetsCollector[] drillSideways, String[] drillSidewaysDims,
+                               IndexState state, JSONArray facetResults) throws IOException {
 
-    final Filter filter;
-    if (r.hasParam("filter")) {
-      filter = parseFilter(timeStamp, r, state, r.getStruct("filter"));
-    } else {
-      filter = null;
+    Map<String,FacetsCollector> dsDimMap = new HashMap<String,FacetsCollector>();
+    if (drillSidewaysDims != null) {
+      for(int i=0;i<drillSidewaysDims.length;i++) {
+        dsDimMap.put(drillSidewaysDims[i], drillSideways[i]);
+      }
     }
 
-    final FacetSearchParams fsp;
+    // Holds already computed Facets, since more
+    // than one dimension can share a single
+    // index field name:
+    Map<String,Facets> indexFieldNameToFacets = new HashMap<String,Facets>();
 
-    final Set<String> facetRequestDims;
+    for(Object o2 : r.getList("facets")) {
+      Request r2 = (Request) o2;
 
-    if (r.hasParam("facets")) {
-      List<FacetRequest> requests = new ArrayList<FacetRequest>();
-      facetRequestDims = new HashSet<String>();
-      for(Object o2 : r.getList("facets")) {
-        Request r2 = (Request) o2;
-
-        if (r2.hasParam("numericRanges")) {
-          FieldDef fd = state.getField(r2, "path");
-          String path = fd.name;
-
-          if (!fd.faceted.equals("numericRange")) {
-            r2.fail("numericRanges", "field \"" + path + "\" was not registered with facet=numericRange");
-          }
-          if (fd.valueType.equals("int") || fd.valueType.equals("long")) {
-            List<LongRange> ranges = new ArrayList<LongRange>();
-            for(Object o : r2.getList("numericRanges")) {
-              Request r3 = (Request) o;
-              ranges.add(new LongRange(r3.getString("label"),
-                                       r3.getLong("min"),
-                                       r3.getBoolean("minInclusive"),
-                                       r3.getLong("max"),
-                                       r3.getBoolean("maxInclusive")));
-            }
-            requests.add(new RangeFacetRequest<LongRange>(path, ranges));
-          } else {
-            r2.fail("numericRanges", "only int/long currently supported");
+      FieldDef fd = state.getField(r2, "dim");
+
+      FacetResult facetResult;
+
+      if (r2.hasParam("numericRanges")) {
+        if (!fd.faceted.equals("numericRange")) {
+          r2.fail("numericRanges", "field \"" + fd.name + "\" was not registered with facet=numericRange");
+        }
+        if (fd.valueType.equals("int") || fd.valueType.equals("long")) {
+          List<Object> rangeList = r2.getList("numericRanges");
+          LongRange[] ranges = new LongRange[rangeList.size()];
+          for(int i=0;i<ranges.length;i++) {
+            Request r3 = (Request) rangeList.get(i);
+            ranges[i] = new LongRange(r3.getString("label"),
+                                      r3.getLong("min"),
+                                      r3.getBoolean("minInclusive"),
+                                      r3.getLong("max"),
+                                      r3.getBoolean("maxInclusive"));
+          }
+
+          FacetsCollector c = dsDimMap.get(fd.name);
+          if (c == null) {
+            c = drillDowns;
           }
+
+          Facets facets = new LongRangeFacetCounts(fd.name,
+                                                   c,
+                                                   ranges);
+          facetResult = facets.getTopChildren(0, fd.name);
         } else {
-          CategoryPath cp;
+          // nocommit float/double too
+          r2.fail("numericRanges", "only int/long currently supported");
+
+          // Dead code but compiler disagrees:
+          facetResult = null;
+        }
+      } else {
+
+        // Taxonomy facets
+        if (fd.faceted.equals("no")) {
+          r2.fail("path", "field \"" + fd.name + "\" was not registered with facet enabled");
+        } else if (fd.faceted.equals("numericRange")) {
+          r2.fail("path", "field \"" + fd.name + "\" was registered with facet=numericRange; must pass numericRanges in the request");
+        }
+
+        String[] path;
+        if (r2.hasParam("path")) {
           if (r2.isString("path")) {
-            String path = r2.getString("path");
-            cp = new CategoryPath(path);
+            path = new String[] {r2.getString("path")};
           } else {
             List<Object> l = r2.getList("path");
             if (l.isEmpty()) {
               r2.fail("path", "path must contain at least one part");
             }
-            String[] path = new String[l.size()];
+            path = new String[l.size()];
             for(int idx=0;idx<path.length;idx++) {
               path[idx] = l.get(idx).toString();
             }
-            cp = new CategoryPath(path);
-          }
-          FieldDef fd = state.getField(cp.components[0]);
-          if (fd.faceted.equals("no")) {
-            r2.fail("path", "field \"" + fd.name + "\" was not registered with facet enabled");
-          } else if (fd.faceted.equals("numericRange")) {
-            r2.fail("path", "field \"" + fd.name + "\" was registered with facet=numericRange; must pass numericRanges in the request");
           }
+        } else {
+          path = new String[0];
+        }
 
-          //System.out.println("cp: " + cp + "; len=" + cp.components.length);
+        Facets facets;
 
-          FacetRequest fr;
-          if (r2.getBoolean("useOrdsCache") == false) {
-            fr = new CountFacetRequest(cp, r2.getInt("topN"));
+        FacetsCollector c = dsDimMap.get(fd.name);
+
+        boolean useCachedOrds = r2.getBoolean("useOrdsCache");
+
+        if (c != null) {
+          // This dimension was used in
+          // drill-down; compute its facet counts from the
+          // drill-sideways collector:
+          if (useCachedOrds) {
+            facets = new TaxonomyFacetCounts(state.ordsCache,
+                                             s.taxonomyReader,
+                                             state.facetsConfig, 
+                                             c);
           } else {
-            fr = new CountFacetRequest(cp, r2.getInt("topN")) {
-              @Override
-              public FacetsAggregator createFacetsAggregator(FacetIndexingParams fip) {
-                return new CachedOrdsCountingFacetsAggregator();
-              }
-            };
+            facets = new FastTaxonomyFacetCounts(s.taxonomyReader,
+                                                 state.facetsConfig, 
+                                                 c);
           }
-          requests.add(fr);
+        } else {
 
-          facetRequestDims.add(cp.components[0]);
+          // See if we already computed facet
+          // counts for this indexFieldName:
+          String indexFieldName = state.facetsConfig.getDimConfig(fd.name).indexFieldName;
+          facets = indexFieldNameToFacets.get(indexFieldName);
+          if (facets == null) {
+            if (useCachedOrds) {
+              facets = new TaxonomyFacetCounts(state.ordsCache,
+                                               s.taxonomyReader,
+                                               state.facetsConfig, 
+                                               c);
+            } else {
+              facets = new FastTaxonomyFacetCounts(s.taxonomyReader,
+                                                   state.facetsConfig, 
+                                                   drillDowns);
+            }
+            indexFieldNameToFacets.put(indexFieldName, facets);
+          }
         }
+
+        facetResult = facets.getTopChildren(r2.getInt("topN"), fd.name, path);
+      }
+
+      if (facetResult == null) {
+        facetResults.add(null);
+      } else {
+        facetResults.add(buildFacetResultJSON(facetResult));
       }
-      fsp = requests.size() == 0 ? null : new FacetSearchParams(state.facetIndexingParams, requests);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public FinishRequest handle(final IndexState state, final Request r, Map<String,List<String>> params) throws Exception {
+
+    state.verifyStarted(r);
+
+    final Map<ToParentBlockJoinQuery,BlockJoinQueryChild> useBlockJoinCollector = new HashMap<ToParentBlockJoinQuery,BlockJoinQueryChild>();
+
+    // App should re-use a previous timestamp if user does a
+    // follow-on action, so that things relying on timestamp
+    // (e.g. dynamic range facet counts, recency blended
+    // sorting) don't change as the user drills down / next
+    // pages / etc.
+    final long timeStamp;
+    if (r.hasParam("timeStamp")) {
+      timeStamp = r.getLong("timeStamp");
     } else {
-      fsp = null;
-      facetRequestDims = null;
+      timeStamp = System.currentTimeMillis()/1000;
+    }
+
+    JSONObject diagnostics = new JSONObject();
+
+    Query q = extractQuery(state, r, timeStamp, useBlockJoinCollector);
+
+    final Filter filter;
+    if (r.hasParam("filter")) {
+      filter = parseFilter(timeStamp, r, state, r.getStruct("filter"));
+    } else {
+      filter = null;
     }
 
     final Set<String> fields;
@@ -1592,852 +1702,553 @@ public class SearchHandler extends Handl
       highlightFields = null;
     }
 
-    final HighlighterConfig highlighter = getHighlighter(state, r, highlightFields);
+    HighlighterConfig highlighter = getHighlighter(state, r, highlightFields);
 
-    final String groupField;
-    final Sort groupSort;
-    final int groupsPerPage;
-    final int hitsPerGroup;
-    final int groupStart;
-    final boolean doGroupMaxScore;
-    final boolean doGroupDocScores;
-    final boolean doTotalGroupCount;
-    final DocValuesType groupDVType;
-
-    if (r.hasParam("grouping")) {
-      if (!useBlockJoinCollector.isEmpty()) {
-        r.fail("grouping", "cannot do both grouping and ToParentBlockJoinQuery with returnChildHits=true");
-      }
-      Request grouping = r.getStruct("grouping");
-      FieldDef fd = state.getField(grouping, "field");
-      groupField = fd.name;
-      groupDVType = fd.fieldType.docValueType();
-      if (groupDVType == null) {
-        grouping.fail("field", "field \"" + groupField + "\" was not registered with group=true");
-      }
-
-      if (grouping.hasParam("sort")) {
-        groupSort = parseSort(timeStamp, state, grouping.getList("sort"));
-      } else {
-        groupSort = Sort.RELEVANCE;
-      }
-      groupsPerPage = grouping.getInt("groupsPerPage");
-      hitsPerGroup = grouping.getInt("hitsPerGroup");
-      doGroupMaxScore = grouping.getBoolean("doMaxScore");
-      doGroupDocScores = grouping.getBoolean("doDocScores");
-      doTotalGroupCount = grouping.getBoolean("doTotalGroupCount");
-      groupStart = grouping.getInt("groupStart");
-      //System.out.println("doGroupMaxScore=" + doGroupMaxScore);
+    diagnostics.put("parsedQuery", q.toString());
 
-    } else {
-      groupField = null;
-      groupSort = null;
-      groupsPerPage = 0;
-      hitsPerGroup = 0;
-      doGroupMaxScore = false;
-      doGroupDocScores = false;
-      groupDVType = null;
-      doTotalGroupCount = false;
-      groupStart = 0;
-    }
-
-    final List<CategoryPath> allDrillDowns = new ArrayList<CategoryPath>();
-
-    // True for pure browse ... we use this only for
-    // browse-only facets caching:
-    final boolean isMatchAll = q instanceof MatchAllDocsQuery && filter == null;
-
-    final Set<String> drillDownDims;
-    final List<CategoryPath[]> drillDowns;
-    final List<NumericRangeQuery<? extends Number>> rangeDrillDowns;
+    TopDocs hits;
+    TopGroups<BytesRef> groups;
+    TopGroups<Integer> joinGroups;
+    int totalGroupCount = -1;
 
-    if (r.hasParam("drillDowns")) {
-      List<Object> drillDownList = r.getList("drillDowns");
-      if (!drillDownList.isEmpty()) {
-        drillDowns = new ArrayList<CategoryPath[]>();
-        rangeDrillDowns = new ArrayList<NumericRangeQuery<? extends Number>>();
-        drillDownDims = new HashSet<String>();
-        for(Object o : drillDownList) {
-          Request fr = (Request) o;
-          FieldDef fd = state.getField(fr, "field");
-          String field = fd.name;
+    final String resultString;
 
-          List<Object> values = fr.getList("values");
-          List<CategoryPath> paths = new ArrayList<CategoryPath>();
-          for(Object o2 : values) {
-            List<String> path = new ArrayList<String>();
-            path.add(field);
-            if (o2 instanceof JSONArray) {
-              for(Object sub : ((JSONArray) o2)) {
-                path.add(sub.toString());
-              }
-            } else {
-              path.add(o2.toString());
-            }
-            if (fd.faceted.equals("numericRange")) {
-              if (values.size() != 1) {
-                fr.fail("values", "numericRange drilldown should have only one value");
-              }
-              if (path.size() != 2) {
-                fr.fail("values", "numericRange drilldown should have only one value");
-              }
-              boolean found = false;
-              for(FacetRequest request : fsp.facetRequests) {
-                if (request.categoryPath.components[0].equals(field)) {
-                  if (!(request instanceof RangeFacetRequest)) {
-                    r.fail("facets", "field \"" + field + "\" must have numericRange facet request");
-                  }
-                  RangeFacetRequest<? extends Range> rfr = (RangeFacetRequest<? extends Range>) request;
-                  for(Range range : rfr.ranges) {
-                    if (range.label.equals(path.get(1))) {
-                      if (range instanceof LongRange) {
-                        LongRange lr = (LongRange) range;
-                        rangeDrillDowns.add(NumericRangeQuery.newLongRange(path.get(0), lr.min, lr.max, lr.minInclusive, lr.maxInclusive));
-                        found = true;
-                        break;
-                      } else {
-                        throw new AssertionError("only Long ranges handled currently");
-                      }
-                    }
-                  }
-                }
-              }
-              if (!found) {
-                fr.fail("field", "could not locate numeric range for \"" + path.get(1) + "\"");
-              }
-            } else {
-              CategoryPath cp = new CategoryPath(path.toArray(new String[path.size()]));
-              allDrillDowns.add(cp);
-              paths.add(cp);
-            }
-          }
-          if (!paths.isEmpty()) {
-            drillDowns.add(paths.toArray(new CategoryPath[paths.size()]));
-            drillDownDims.add(field);
-          }
-        }
-      } else {
-        drillDownDims = null;
-        drillDowns = null;
-        rangeDrillDowns = null;
-      }
-    } else {
-      drillDownDims = null;
-      drillDowns = null;
-      rangeDrillDowns = null;
-    }
+    // Pull the searcher we will use
+    final SearcherAndTaxonomy s = getSearcherAndTaxonomy(r, state, diagnostics);
 
-    diagnostics.put("parsedQuery", q.toString());
+    // matching finally clause releases this searcher:
+    try {
 
-    final int lastDocID;
-    final float lastScore;
-    final List<Object> lastFieldValues;
-    if (r.hasParam("searchAfter")) {
-      if (groupField != null) {
-        r.fail("searchAfter", "cannot use searchAfter with grouping");
-      }
-      if (!useBlockJoinCollector.isEmpty()) {
-        r.fail("searchAfter", "cannot use searchAfter with ToParentBlockJoinQuery with returnChildHits=true");
-      }
+      q = s.searcher.rewrite(q);
+      //System.out.println("after rewrite: " + q);
+      diagnostics.put("rewrittenQuery", q.toString());
 
-      Request sa = r.getStruct("searchAfter");
-      if (sa.hasParam("lastDoc")) {
-        lastDocID = sa.getInt("lastDoc");
-        if (sort != null) {
-          lastFieldValues = sa.getList("lastFieldValues");
-          if (lastFieldValues.size() != sort.getSort().length) {
-            sa.fail("lastFieldValues", "length of lastFieldValues must be the same as number of sort fields; be sure to pass the previous page's searchState.lastFieldValues");
-          }
-          lastScore = 0.0f;
-        } else {
-          lastFieldValues = null;
-          lastScore = sa.getFloat("lastScore");
-        }
-      } else {
-        lastDocID = -1;
-        lastScore = 0.0f;
-        lastFieldValues = null;
+      if (filter != null) {
+        q = new FilteredQuery(q, filter);
       }
-    } else {
-      lastDocID = -1;
-      lastScore = 0.0f;
-      lastFieldValues = null;
-    }
 
-    return new FinishRequest() {
+      // nocommit add test with drill down on OR of fields:
 
-      @Override
-      public String finish() throws IOException {
-        // nocommit allow passing in prior search state &
-        // using prior searcher
+      // TODO: re-enable this?  else we never get
+      // in-order collectors
+      //Weight w = s.createNormalizedWeight(q2);
 
-        List<FacetResult> facets;
-        TopDocs hits;
-        TopGroups<BytesRef> groups;
-        TopGroups<Integer> joinGroups;
-        int totalGroupCount = -1;
-        final Map<String,FacetArrays> dimFacetArrays = new HashMap<String,FacetArrays>();
-
-        // Pull the searcher we will use
-        final SearcherAndTaxonomy s = getSearcherAndTaxonomy(r, state, searcherVersion, searcherSnapshot, diagnostics);
-        long searcherToken = ((DirectoryReader) s.searcher.getIndexReader()).getVersion();
+      DrillDownQuery ddq = addDrillDowns(timeStamp, state, r, q);
 
-        try {
+      diagnostics.put("drillDownQuery", q.toString());
 
-          Query q2 = s.searcher.rewrite(q);
-          //System.out.println("after rewrite: " + q2);
-          diagnostics.put("rewrittenQuery", q2.toString());
-
-          if (filter != null) {
-            q2 = new FilteredQuery(q2, filter);
-          }
-
-          // TODO: re-enable this?  else we never get
-          // in-order collectors
-          //Weight w = s.createNormalizedWeight(q2);
-
-          // If there are any drill-downs, wrap in
-          // DrillDownQuery:
-          DrillDownQuery ddq = new DrillDownQuery(state.facetIndexingParams, q2);
-          if (drillDowns != null) {
-            for(CategoryPath[] dd : drillDowns) {
-              ddq.add(dd);
-            }
-          }
-          if (rangeDrillDowns != null) {
-            for(NumericRangeQuery nrq : rangeDrillDowns) {
-              ddq.add(nrq.getField(), nrq);
-            }
-          }
-          q2 = ddq;
-          diagnostics.put("drillDownQuery", q2.toString());
-
-          Collector c;
-          TermFirstPassGroupingCollector groupCollector = null;
-          TermAllGroupsCollector allGroupsCollector = null;
-          if (groupField != null) {
-            groupCollector = new TermFirstPassGroupingCollector(groupField, groupSort, groupsPerPage);
-            if (doTotalGroupCount) {
-              allGroupsCollector = new TermAllGroupsCollector(groupField);
-              c = MultiCollector.wrap(groupCollector, allGroupsCollector);
-            } else {
-              c = groupCollector;
-            }
-          } else if (!useBlockJoinCollector.isEmpty()) {
-            Iterator<Map.Entry<ToParentBlockJoinQuery,BlockJoinQueryChild>> it = useBlockJoinCollector.entrySet().iterator();
-            Map.Entry<ToParentBlockJoinQuery,BlockJoinQueryChild> ent = it.next();
-            BlockJoinQueryChild child = ent.getValue();
-            c = new ToParentBlockJoinCollector(sort == null ? Sort.RELEVANCE : sort,
-                                               topHits, child.trackScores, child.trackMaxScore);
-          } else if (sort == null) {
-            ScoreDoc searchAfter;
-            if (lastDocID != -1) {
-              searchAfter = new ScoreDoc(lastDocID, lastScore);
-            } else {
-              searchAfter = null;
-            }
-            //c = TopScoreDocCollector.create(topHits, searchAfter, !w.scoresDocsOutOfOrder());
-            c = TopScoreDocCollector.create(topHits, searchAfter, false);
-          } else {
-            FieldDoc searchAfter;
-            if (lastDocID != -1) {
-              searchAfter = new FieldDoc(lastDocID, lastScore, lastFieldValues.toArray(new Object[lastFieldValues.size()]));
-            } else {
-              searchAfter = null;
-            }
-            //c = TopFieldCollector.create(sort, topHits, searchAfter, true, doDocScores, doMaxScore, !w.scoresDocsOutOfOrder());
-            c = TopFieldCollector.create(sort, topHits, searchAfter, true, doDocScores, doMaxScore, false);
-          }
+      Collector c;
+      TermFirstPassGroupingCollector groupCollector = null;
+      TermAllGroupsCollector allGroupsCollector = null;
 
-          DrillSideways.DrillSidewaysResult dsResults;
+      FieldDef groupField = null;
+      Request grouping = null;
+      Sort groupSort = null;
+      Sort sort;
+      Request sortRequest;
+      if (r.hasParam("sort")) {
+        sortRequest = r.getStruct("sort");
+        sort = parseSort(timeStamp, state, sortRequest.getList("fields"));
+      } else {
+        sortRequest = null;
+        sort = null;
+      }
 
-          // Careful top-level facets cache logic:
-          Map<FacetRequest,FacetResult> cachedFacetResults = null;
+      int topHits = r.getInt("topHits");
 
-          // nocommit can we do better?  sometimes downgrade
-          // to DDQ not DS?
-          // nocommit improve this: we can use the cache if
-          // there's a single DS dim, for just that dim
-          // (other dims must recompute)
-          // nocommit turn this back on but ... this causes
-          // NPEs when drillDownFacet tries to fill
+      if (r.hasParam("grouping")) {
+        if (r.hasParam("searchAfter")) {
+          r.fail("searchAfter", "cannot use searchAfter with grouping");
+        }
+        if (!useBlockJoinCollector.isEmpty()) {
+          r.fail("grouping", "cannot do both grouping and ToParentBlockJoinQuery with returnChildHits=true");
+        }
+        grouping = r.getStruct("grouping");
+        groupField = state.getField(grouping, "field");
 
-          FacetSearchParams fsp2;
+        // Make sure the group-by field was indexed with group=true:
+        if (groupField.fieldType.docValueType() == null) {
+          grouping.fail("field", "field \"" + groupField.name + "\" was not registered with group=true");
+        }
 
-          JSONArray cachedDims = new JSONArray();
-          diagnostics.put("facetCachedDims", cachedDims);
-          //System.out.println("isMatchAll=" + isMatchAll);
+        if (grouping.hasParam("sort")) {
+          groupSort = parseSort(timeStamp, state, grouping.getList("sort"));
+        } else {
+          groupSort = Sort.RELEVANCE;
+        }
 
-          if (isMatchAll && fsp != null && (drillDowns == null || drillDowns.size() == 1)) {
-            TopFacetsCache cache = ((MyIndexSearcher) s.searcher).topFacetsCache;
-            cachedFacetResults = new HashMap<FacetRequest,FacetResult>();
+        groupCollector = new TermFirstPassGroupingCollector(groupField.name, groupSort, grouping.getInt("groupsPerPage"));
+        if (grouping.getBoolean("doTotalGroupCount")) {
+          allGroupsCollector = new TermAllGroupsCollector(groupField.name);
+          c = MultiCollector.wrap(groupCollector, allGroupsCollector);
+        } else {
+          c = groupCollector;
+        }
+      } else if (useBlockJoinCollector.isEmpty() == false) {
+        if (r.hasParam("searchAfter")) {
+          r.fail("searchAfter", "cannot use searchAfter with ToParentBlockJoinQuery with returnChildHits=true");
+        }
+        Iterator<Map.Entry<ToParentBlockJoinQuery,BlockJoinQueryChild>> it = useBlockJoinCollector.entrySet().iterator();
+        Map.Entry<ToParentBlockJoinQuery,BlockJoinQueryChild> ent = it.next();
+        BlockJoinQueryChild child = ent.getValue();
+        c = new ToParentBlockJoinCollector(sort == null ? Sort.RELEVANCE : sort,
+                                           topHits, child.trackScores, child.trackMaxScore);
+      } else if (sort == null) {
+        ScoreDoc searchAfter;
+        if (r.hasParam("searchAfter")) {
+          Request sa = r.getStruct("searchAfter");
+          searchAfter = new ScoreDoc(sa.getInt("lastDoc"),
+                                     sa.getFloat("lastScore"));
+        } else {
+          searchAfter = null;
+        }
+        //c = TopScoreDocCollector.create(topHits, searchAfter, !w.scoresDocsOutOfOrder());
+        c = TopScoreDocCollector.create(topHits, searchAfter, false);
+      } else {
+        // Sort by fields:
+        FieldDoc searchAfter;
+        if (r.hasParam("searchAfter")) {
+          Request sa = r.getStruct("searchAfter");
+          List<Object> lastFieldValues = sa.getList("lastFieldValues");
+          searchAfter = new FieldDoc(sa.getInt("lastDoc"),
+                                     0.0f,
+                                     lastFieldValues.toArray(new Object[lastFieldValues.size()]));
+        } else {
+          searchAfter = null;
+        }
 
-            List<FacetRequest> unCachedRequests = new ArrayList<FacetRequest>();
+        //c = TopFieldCollector.create(sort, topHits, searchAfter, true, doDocScores, doMaxScore, !w.scoresDocsOutOfOrder());
+        c = TopFieldCollector.create(sort, topHits, searchAfter, true,
+                                     sortRequest.getBoolean("doDocScores"),
+                                     sortRequest.getBoolean("doMaxScore"),
+                                     false);
+      }
 
-            for (FacetRequest req : fsp.facetRequests) {
-              if (req instanceof RangeFacetRequest) {
-                unCachedRequests.add(req);
-                continue;
-              }
-              String dim = req.categoryPath.components[0];
-              boolean doCache = false;
-              if (drillDowns == null || drillDownDims.contains(dim)) {
-                // This dimension is the "browse only" facet
-                // counts, so we can consult cache:
-                FacetResult result = cache.get(req);
-                cachedDims.add(dim);
-                if (result != null) {
-                  Set<Integer> ords = new HashSet<Integer>();
-                  for(FacetResultNode childNode : result.getFacetResultNode().subResults) {
-                    ords.add(childNode.ordinal);
-                  }
-                  doCache = true;
-                  for(CategoryPath cp : allDrillDowns) {
-                    if (cp.components[0].equals(dim)) {
-                      int ord = s.taxonomyReader.getOrdinal(cp);
-                      if (ord == -1) {
-                        r.fail("drillDowns", "path " + cp + " does not exist");
-                      }
-                      if (!ords.contains(ord)) {
-                        // Cannot use cache: there is an
-                        // explicit drill down on this dim
-                        // that was not counted in the
-                        // cached results:
-                        doCache = false;
-                        break;
-                      }
-                    }
-                  }
-                  if (doCache) {
-                    cachedFacetResults.put(req, result);
-                  }
-                }
-              }
-              if (!doCache) {
-                unCachedRequests.add(req);
-              }
-            }
+      // nocommit can we do better?  sometimes downgrade
+      // to DDQ not DS?
+      // nocommit improve this: we can use the cache if
+      // there's a single DS dim, for just that dim
+      // (other dims must recompute)
+      // nocommit turn this back on but ... this causes
+      // NPEs when drillDownFacet tries to fill
 
-            if (!cachedFacetResults.isEmpty()) {
-              // Make a new FacetSearchParams:
-              if (unCachedRequests.isEmpty()) {
-                fsp2 = null;
-              } else {
-                fsp2 = new FacetSearchParams(state.facetIndexingParams, unCachedRequests);
-              }
-            } else {
-              // No dimensions were cached:
-              fsp2 = fsp;
-            }
-          } else {
-            // Browse only facet cache does not apply
-            // (e.g. because there are 2 drill downs, or
-            // because query is not MatchAll):
-            fsp2 = fsp;
-          }
-
-          //System.out.println("useCache=" + useFacetsCache + " cacheHit=" + facetsCacheHit);
-
-          long searchStartTime = System.nanoTime();
-
-          if (fsp2 != null) {
-            // Use DrillSideways to do the search, so we get
-            // sideways counts
-            DrillSideways ds = new DrillSideways(s.searcher, s.taxonomyReader) {
-                @Override
-                protected FacetsAccumulator getDrillDownAccumulator(FacetSearchParams fsp) {
-                  FacetArrays arrays = new FacetArrays(s.taxonomyReader.getSize());
-                  dimFacetArrays.put(null, arrays);
-                  // nocommit: cutover to lucene's
-                  return MultiFacetsAccumulator.create(fsp, searcher.getIndexReader(), s.taxonomyReader, arrays);
-                }
+      //System.out.println("useCache=" + useFacetsCache + " cacheHit=" + facetsCacheHit);
 
-                @Override
-                protected FacetsAccumulator getDrillSidewaysAccumulator(String dim, FacetSearchParams fsp) {
-                  FacetArrays arrays = new FacetArrays(s.taxonomyReader.getSize());
-                  dimFacetArrays.put(dim, arrays);
-                  // nocommit: cutover to lucene's
-                  return MultiFacetsAccumulator.create(fsp, searcher.getIndexReader(), s.taxonomyReader, arrays);
-                }
+      long searchStartTime = System.nanoTime();
 
-                @Override
-                protected boolean scoreSubDocsAtOnce() {
-                  // If we are using
-                  // ToParentBlockJoinCollector then all
-                  // sub-docs must be scored at once:
-                  return !useBlockJoinCollector.isEmpty();
-                }
-              };
+      // Holds the search result JSON object:
+      JSONObject result = new JSONObject();
+      result.put("diagnostics", diagnostics);
 
-            dsResults = ds.search((DrillDownQuery) q2, c, fsp2);
-          } else {
-            //((MyIndexSearcher) s).search(w, c2);
-            s.searcher.search(q2, c);
-            dsResults = null;
-          }
-          diagnostics.put("firstPassSearchMS", ((System.nanoTime()-searchStartTime)/1000000.0));
-
-          if (groupField != null) {
-            Collection<SearchGroup<BytesRef>> topGroups = groupCollector.getTopGroups(groupStart, true);
-            if (topGroups != null) {
-              TermSecondPassGroupingCollector c3 = new TermSecondPassGroupingCollector(groupField,
-                                                                                       topGroups,
-                                                                                       groupSort,
-                                                                                       sort,
-                                                                                       hitsPerGroup,
-                                                                                       doGroupDocScores,
-                                                                                       doGroupMaxScore,
-                                                                                       true);
-              long t0 = System.nanoTime();
-              // TODO: should we ... pre-rewrite a query if
-              // we know we will need to do 2 passes?
-              //((MyIndexSearcher) s).search(w, c3);
-              s.searcher.search(q2, c3);
-              diagnostics.put("secondPassSearchMS", ((System.nanoTime()-t0)/1000000));
-
-              groups = c3.getTopGroups(0);
-              hits = null;
-              joinGroups = null;
-              if (allGroupsCollector != null) {
-                totalGroupCount = allGroupsCollector.getGroups().size();
-              }
-            } else {
-              hits = null;
-              groups = null;
-              joinGroups = null;
-              totalGroupCount = 0;
-            }
-          } else if (!useBlockJoinCollector.isEmpty()) {
-            assert useBlockJoinCollector.size() == 1;
-            Iterator<Map.Entry<ToParentBlockJoinQuery,BlockJoinQueryChild>> it = useBlockJoinCollector.entrySet().iterator();
-            Map.Entry<ToParentBlockJoinQuery,BlockJoinQueryChild> ent = it.next();
-            BlockJoinQueryChild child = ent.getValue();
-
-            joinGroups = ((ToParentBlockJoinCollector) c).getTopGroups(ent.getKey(),
-                                                                       child.sort, startHit,
-                                                                       child.maxChildren, 0, true);
-            groups = null;
-            hits = null;
-          } else {
-            groups = null;
-            joinGroups = null;
-            hits = ((TopDocsCollector) c).topDocs();
-
-            if (startHit != 0) {
-              // Slice:
-              int count = Math.max(0, hits.scoreDocs.length - startHit);
-              ScoreDoc[] newScoreDocs = new ScoreDoc[count];
-              if (count > 0) {
-                System.arraycopy(hits.scoreDocs, startHit, newScoreDocs, 0, count);
-              }
-              hits = new TopDocs(hits.totalHits,
-                                 newScoreDocs,
-                                 hits.getMaxScore());
-            }
-          }
+      final IndexState indexState = state;
 
-          if (fsp != null) {
-            if (dsResults != null) {
-              facets = dsResults.facetResults;
-            } else {
-              facets = null;
-            }
+      if (r.hasParam("facets")) {
 
-            // nocommit disabled top facets cache for testing
-            if (false && isMatchAll && drillDowns == null && facets != null) {
-              // Only cache on pure browse case, and don't
-              // cache dynamic numeric ranges:
-              TopFacetsCache cache = ((MyIndexSearcher) s.searcher).topFacetsCache;
-              for(FacetResult fr : facets) {
-                if (!(fr.getFacetRequest() instanceof RangeFacetRequest) && fr.getFacetRequest().numResults < TOP_FACET_CACHE_MAX_FACET_COUNT) {
-                  cache.add(fr.getFacetRequest(), fr);
-                }
+        final JSONArray facetResults = new JSONArray();
+        result.put("facets", facetResults);
+
+        // Always use drill sideways; it downgrades to a
+        // "normal" query if there were no drilldowns:
+        DrillSideways ds = new DrillSideways(s.searcher, state.facetsConfig, s.taxonomyReader) {
+
+            private FacetsCollector getCollector(String dim, Map<String,FacetsCollector> dsMap, FacetsCollector drillDowns) {
+              FacetsCollector c = dsMap.get(dim);
+              if (c == null) {
+                c = drillDowns;
               }
+
+              return c;
             }
-          } else {
-            facets = null;
-          }
 
-          int[] highlightDocIDs = null;
-          if (groupField != null) {
-            if (groups != null) {
-
-              // These groups are already sliced according
-              // to groupStart:
-              int count = 0;
-              for(GroupDocs<BytesRef> group : groups.groups) {
-                count += group.scoreDocs.length;
-              }
-              if (count > 0) {
-                highlightDocIDs = new int[count];
+            @Override
+            protected Facets buildFacetsResult(FacetsCollector drillDowns, FacetsCollector[] drillSideways, String[] drillSidewaysDims) throws IOException {
+              fillFacetResults(r, s, drillDowns, drillSideways, drillSidewaysDims, indexState, facetResults);
+              return null;
+            }
 
-                int upto = 0;
-                for(GroupDocs<BytesRef> group : groups.groups) {
-                  for(ScoreDoc scoreDoc : group.scoreDocs) {
-                    highlightDocIDs[upto++] = scoreDoc.doc;
-                  }
-                }
-              }
+            @Override
+            protected boolean scoreSubDocsAtOnce() {
+              // If we are using
+              // ToParentBlockJoinCollector then all
+              // sub-docs must be scored at once:
+              return !useBlockJoinCollector.isEmpty();
             }
-          } else if (!useBlockJoinCollector.isEmpty()) {
-            if (joinGroups != null) {
+          };
 
-              int count = 0;
-              for(GroupDocs<Integer> group : joinGroups.groups) {
-                // for the parent docID:
-                count++;
-                // for all child docs:
-                count += group.scoreDocs.length;
-              }
+        // Fills in facetResults as a side-effect:
+        ds.search(ddq, c);
+      } else {
+        s.searcher.search(ddq, c);
+      }
 
-              if (count > 0) {
-                highlightDocIDs = new int[count];
+      diagnostics.put("firstPassSearchMS", ((System.nanoTime()-searchStartTime)/1000000.0));
 
-                int upto = 0;
-                for(GroupDocs<Integer> group : joinGroups.groups) {
-                  highlightDocIDs[upto++] = group.groupValue.intValue();
-                  for(ScoreDoc scoreDoc : group.scoreDocs) {
-                    highlightDocIDs[upto++] = scoreDoc.doc;
-                  }
-                }
-              }
-            }
-          } else {
-            highlightDocIDs = new int[hits.scoreDocs.length];
-            for(int i=0;i<hits.scoreDocs.length;i++) {
-              highlightDocIDs[i] = hits.scoreDocs[i].doc;
-            }
+      int startHit = r.getInt("startHit");
+
+      if (grouping != null) {
+        Collection<SearchGroup<BytesRef>> topGroups = groupCollector.getTopGroups(grouping.getInt("groupStart"), true);
+        if (topGroups != null) {
+          TermSecondPassGroupingCollector c3 = new TermSecondPassGroupingCollector(groupField.name,
+                                                                                   topGroups,
+                                                                                   groupSort,
+                                                                                   sort,
+                                                                                   grouping.getInt("hitsPerGroup"),
+                                                                                   grouping.getBoolean("doDocScores"),
+                                                                                   grouping.getBoolean("doMaxScore"),
+                                                                                   true);
+          long t0 = System.nanoTime();
+          //((MyIndexSearcher) s).search(w, c3);
+          s.searcher.search(ddq, c3);
+          diagnostics.put("secondPassSearchMS", ((System.nanoTime()-t0)/1000000));
+
+          groups = c3.getTopGroups(0);
+          hits = null;
+          joinGroups = null;
+          if (allGroupsCollector != null) {
+            totalGroupCount = allGroupsCollector.getGroups().size();
           }
+        } else {
+          hits = null;
+          groups = null;
+          joinGroups = null;
+          totalGroupCount = 0;
+        }
+      } else if (!useBlockJoinCollector.isEmpty()) {
+        assert useBlockJoinCollector.size() == 1;
+        Iterator<Map.Entry<ToParentBlockJoinQuery,BlockJoinQueryChild>> it = useBlockJoinCollector.entrySet().iterator();
+        Map.Entry<ToParentBlockJoinQuery,BlockJoinQueryChild> ent = it.next();
+        BlockJoinQueryChild child = ent.getValue();
+
+        joinGroups = ((ToParentBlockJoinCollector) c).getTopGroups(ent.getKey(),
+                                                                   child.sort, startHit,
+                                                                   child.maxChildren, 0, true);
+        groups = null;
+        hits = null;
+      } else {
+        groups = null;
+        joinGroups = null;
+        hits = ((TopDocsCollector) c).topDocs();
+
+        if (startHit != 0) {
+          // Slice:
+          int count = Math.max(0, hits.scoreDocs.length - startHit);
+          ScoreDoc[] newScoreDocs = new ScoreDoc[count];
+          if (count > 0) {
+            System.arraycopy(hits.scoreDocs, startHit, newScoreDocs, 0, count);
+          }
+          hits = new TopDocs(hits.totalHits,
+                             newScoreDocs,
+                             hits.getMaxScore());
+        }
+      }
 
-          Map<String,String[]> highlights = null;
+      int[] highlightDocIDs = null;
+      if (groupField != null) {
+        if (groups != null) {
+
+          // These groups are already sliced according
+          // to groupStart:
+          int count = 0;
+          for(GroupDocs<BytesRef> group : groups.groups) {
+            count += group.scoreDocs.length;
+          }
+          if (count > 0) {
+            highlightDocIDs = new int[count];
 
-          long t0 = System.nanoTime();
-          if (highlightDocIDs != null && highlightFields != null && !highlightFields.isEmpty()) {
-            int[] maxPassages = new int[highlightFields.size()];
-            Arrays.fill(maxPassages, highlighter.maxPassages);
-            String[] fields = new String[highlightFields.size()];
             int upto = 0;
-            for(Map.Entry<String,FieldHighlightConfig> ent : highlightFields.entrySet()) {
-              fields[upto] = ent.getKey();
-              FieldHighlightConfig perField = ent.getValue();
-              if (perField.maxPassages != -1) {
-                maxPassages[upto] = perField.maxPassages;
+            for(GroupDocs<BytesRef> group : groups.groups) {
+              for(ScoreDoc scoreDoc : group.scoreDocs) {
+                highlightDocIDs[upto++] = scoreDoc.doc;
               }
-              upto++;
             }
-
-            highlights = highlighter.highlighter.highlightFields(fields,
-                                                                 q,
-                                                                 s.searcher,
-                                                                 highlightDocIDs,
-                                                                 maxPassages);
           }
-          diagnostics.put("highlightTimeMS", (System.nanoTime() - t0)/1000000.);
+        }
+      } else if (!useBlockJoinCollector.isEmpty()) {
+        if (joinGroups != null) {
 
-          JSONObject o = new JSONObject();
-          o.put("diagnostics", diagnostics);
-          t0 = System.nanoTime();
-        
-          if (groupField != null) {
-            if (groups == null) {
-              o.put("totalHits", 0);
-              o.put("totalGroupCount", 0);
-            } else {
-              o.put("totalHits", groups.totalHitCount);
-              o.put("totalGroupedHits", groups.totalGroupedHitCount);
-              if (groups.totalGroupCount != null) {
-                o.put("totalGroupCount", groups.totalGroupCount);
-              } else if (totalGroupCount != -1) {
-                o.put("totalGroupCount", totalGroupCount);
-              }
+          int count = 0;
+          for(GroupDocs<Integer> group : joinGroups.groups) {
+            // for the parent docID:
+            count++;
+            // for all child docs:
+            count += group.scoreDocs.length;
+          }
 
-              // nocommit why am I getting a maxScore back when
-              // I didn't ask for it ... oh because I'm sorting
-              // by relevance ... hmm ... must test field sort
-              // case
+          if (count > 0) {
+            highlightDocIDs = new int[count];
 
-              if (!Float.isNaN(groups.maxScore)) {
-                o.put("maxScore", groups.maxScore);
+            int upto = 0;
+            for(GroupDocs<Integer> group : joinGroups.groups) {
+              highlightDocIDs[upto++] = group.groupValue.intValue();
+              for(ScoreDoc scoreDoc : group.scoreDocs) {
+                highlightDocIDs[upto++] = scoreDoc.doc;
               }
+            }
+          }
+        }
+      } else {
+        highlightDocIDs = new int[hits.scoreDocs.length];
+        for(int i=0;i<hits.scoreDocs.length;i++) {
+          highlightDocIDs[i] = hits.scoreDocs[i].doc;
+        }
+      }
 
-              JSONArray o2 = new JSONArray();
-              o.put("groups", o2);
-              int hitIndex = 0;
-              for(GroupDocs<BytesRef> group : groups.groups) {
-                JSONObject o3 = new JSONObject();
-                o2.add(o3);
-                Object v = group.groupValue;
-                if (v instanceof BytesRef) {
-                  o3.put("groupValue", ((BytesRef) v).utf8ToString());
-                } else {
-                  o3.put("groupValue", v);
-                }
-                o3.put("totalHits", group.totalHits);
-
-                if (!Float.isNaN(group.maxScore)) {
-                  o3.put("maxScore", group.maxScore);
-                }
-
-                if (!Float.isNaN(group.score)) {
-                  o3.put("score", group.score);
-                }
+      Map<String,String[]> highlights = null;
 
-                JSONObject o4 = new JSONObject();
-                o3.put("groupSortFields", o4);
-                SortField[] groupSortFields = groupSort.getSort();
-                for(int i=0;i<groupSortFields.length;i++) {
-                  String field = groupSortFields[i].getField();
-                  if (field == null) {
-                    field = "<score>";
-                  }
-                  o4.put(field, group.groupSortValues[i]);
-                }
+      long t0 = System.nanoTime();
+      if (highlightDocIDs != null && highlightFields != null && !highlightFields.isEmpty()) {
+        int[] maxPassages = new int[highlightFields.size()];
+        Arrays.fill(maxPassages, highlighter.maxPassages);
+        String[] fieldsArray = new String[highlightFields.size()];
+        int upto = 0;
+        for(Map.Entry<String,FieldHighlightConfig> ent : highlightFields.entrySet()) {
+          fieldsArray[upto] = ent.getKey();
+          FieldHighlightConfig perField = ent.getValue();
+          if (perField.maxPassages != -1) {
+            maxPassages[upto] = perField.maxPassages;
+          }
+          upto++;
+        }
 
-                JSONArray o5 = new JSONArray();
-                o3.put("hits", o5);
+        highlights = highlighter.highlighter.highlightFields(fieldsArray,
+                                                             q,
+                                                             s.searcher,
+                                                             highlightDocIDs,
+                                                             maxPassages);
+      }
+      diagnostics.put("highlightTimeMS", (System.nanoTime() - t0)/1000000.);
 
-                for(ScoreDoc hit : group.scoreDocs) {
-                  JSONObject o6 = new JSONObject();
-                  o5.add(o6);
-                  o6.put("doc", hit.doc);
-                  if (!Float.isNaN(hit.score)) {
-                    o6.put("score", hit.score);
-                  }
+      t0 = System.nanoTime();
+        
+      if (groupField != null) {
+        if (groups == null) {
+          result.put("totalHits", 0);
+          result.put("totalGroupCount", 0);
+        } else {
+          result.put("totalHits", groups.totalHitCount);
+          result.put("totalGroupedHits", groups.totalGroupedHitCount);
+          if (groups.totalGroupCount != null) {
+            result.put("totalGroupCount", groups.totalGroupCount);
+          } else if (totalGroupCount != -1) {
+            result.put("totalGroupCount", totalGroupCount);
+          }
+
+          // nocommit why am I getting a maxScore back when
+          // I didn't ask for it ... oh because I'm sorting
+          // by relevance ... hmm ... must test field sort
+          // case
+
+          if (!Float.isNaN(groups.maxScore)) {
+            result.put("maxScore", groups.maxScore);
+          }
+
+          JSONArray o2 = new JSONArray();
+          result.put("groups", o2);
+          int hitIndex = 0;
+          for(GroupDocs<BytesRef> group : groups.groups) {
+            JSONObject o3 = new JSONObject();
+            o2.add(o3);
+            Object v = group.groupValue;
+            if (v instanceof BytesRef) {
+              o3.put("groupValue", ((BytesRef) v).utf8ToString());
+            } else {
+              o3.put("groupValue", v);
+            }
+            o3.put("totalHits", group.totalHits);
 
-                  if (fields != null || highlightFields != null) {
-                    JSONObject o7 = new JSONObject();
-                    o6.put("fields", o7);
-                    fillFields(state, highlighter, s.searcher, o7, hit, fields, highlights, hitIndex, sort);
-                  }
+            if (!Float.isNaN(group.maxScore)) {
+              o3.put("maxScore", group.maxScore);
+            }
 
-                  hitIndex++;
-                }
-              }
+            if (!Float.isNaN(group.score)) {
+              o3.put("score", group.score);
             }
-          } else if (!useBlockJoinCollector.isEmpty()) {
-            // ToParentBlockJoin
-            if (joinGroups == null) {
-              o.put("totalHits", 0);
-              o.put("totalGroupCount", 0);
-            } else {
 
-              assert useBlockJoinCollector.size() == 1;
-              Iterator<Map.Entry<ToParentBlockJoinQuery,BlockJoinQueryChild>> it = useBlockJoinCollector.entrySet().iterator();
-              Map.Entry<ToParentBlockJoinQuery,BlockJoinQueryChild> ent = it.next();
-              BlockJoinQueryChild child = ent.getValue();
-
-              o.put("totalHits", joinGroups.totalHitCount);
-              o.put("totalGroupedHits", joinGroups.totalGroupedHitCount);
-              if (joinGroups.totalGroupCount != null) {
-                o.put("totalGroupCount", joinGroups.totalGroupCount);
+            JSONObject o4 = new JSONObject();
+            o3.put("groupSortFields", o4);
+            SortField[] groupSortFields = groupSort.getSort();
+            for(int i=0;i<groupSortFields.length;i++) {
+              String field = groupSortFields[i].getField();
+              if (field == null) {
+                field = "<score>";
               }
+              o4.put(field, group.groupSortValues[i]);
+            }
 
-              // nocommit why am I getting a maxScore back when
-              // I didn't ask for it ... oh because I'm sorting
-              // by relevance ... hmm ... must test field sort
-              // case
+            JSONArray o5 = new JSONArray();
+            o3.put("hits", o5);
 
-              if (!Float.isNaN(joinGroups.maxScore)) {
-                o.put("maxScore", joinGroups.maxScore);
+            for(ScoreDoc hit : group.scoreDocs) {
+              JSONObject o6 = new JSONObject();
+              o5.add(o6);
+              o6.put("doc", hit.doc);
+              if (!Float.isNaN(hit.score)) {
+                o6.put("score", hit.score);
               }
 
-              JSONArray o2 = new JSONArray();
-              o.put("groups", o2);
-              int hitIndex = 0;
-              for(GroupDocs<Integer> group : joinGroups.groups) {
-                JSONObject o3 = new JSONObject();
-                o2.add(o3);
-                if (fields != null || highlightFields != null) {
-                  JSONObject o4 = new JSONObject();
-                  o3.put("fields", o4);
-                  // nocommit where does parent score come
-                  // from ...
-                  ScoreDoc sd = new ScoreDoc(group.groupValue.intValue(), 0.0f);
-                  fillFields(state, highlighter, s.searcher, o4, sd, fields, highlights, hitIndex, sort);
-                }
-                hitIndex++;
-
-                o3.put("totalHits", group.totalHits);
-
-                if (!Float.isNaN(group.maxScore)) {
-                  o3.put("maxScore", group.maxScore);
-                }
-
-                JSONObject o4 = new JSONObject();
-                o3.put("groupSortFields", o4);
-                SortField[] groupSortFields = (child.sort == null ? Sort.RELEVANCE : child.sort).getSort();
-                for(int i=0;i<groupSortFields.length;i++) {
-                  String field = groupSortFields[i].getField();
-                  if (field == null) {
-                    field = "<score>";
-                  }
-                  o4.put(field, group.groupSortValues[i]);
-                }
-
-                JSONArray o5 = new JSONArray();
-                o3.put("hits", o5);
-
-                for(ScoreDoc hit : group.scoreDocs) {
-                  JSONObject o6 = new JSONObject();
-                  o5.add(o6);
-                  o6.put("doc", hit.doc);
-                  if (!Float.isNaN(hit.score)) {
-                    o6.put("score", hit.score);
-                  }
+              if (fields != null || highlightFields != null) {
+                JSONObject o7 = new JSONObject();
+                o6.put("fields", o7);
+                fillFields(state, highlighter, s.searcher, o7, hit, fields, highlights, hitIndex, sort);
+              }
 
-                  if (fields != null || highlightFields != null) {
-                    JSONObject o7 = new JSONObject();
-                    o6.put("fields", o7);
-                    fillFields(state, highlighter, s.searcher, o7, hit, fields, highlights, hitIndex, child.sort);
-                  }
+              hitIndex++;
+            }
+          }
+        }
+      } else if (!useBlockJoinCollector.isEmpty()) {
+        // ToParentBlockJoin
+        if (joinGroups == null) {
+          result.put("totalHits", 0);
+          result.put("totalGroupCount", 0);
+        } else {
 
-                  hitIndex++;
-                }
+          assert useBlockJoinCollector.size() == 1;
+          Iterator<Map.Entry<ToParentBlockJoinQuery,BlockJoinQueryChild>> it = useBlockJoinCollector.entrySet().iterator();
+          Map.Entry<ToParentBlockJoinQuery,BlockJoinQueryChild> ent = it.next();
+          BlockJoinQueryChild child = ent.getValue();
+
+          result.put("totalHits", joinGroups.totalHitCount);
+          result.put("totalGroupedHits", joinGroups.totalGroupedHitCount);
+          if (joinGroups.totalGroupCount != null) {
+            result.put("totalGroupCount", joinGroups.totalGroupCount);
+          }
+
+          // nocommit why am I getting a maxScore back when
+          // I didn't ask for it ... oh because I'm sorting
+          // by relevance ... hmm ... must test field sort
+          // case
+
+          if (!Float.isNaN(joinGroups.maxScore)) {
+            result.put("maxScore", joinGroups.maxScore);
+          }
+
+          JSONArray o2 = new JSONArray();
+          result.put("groups", o2);
+          int hitIndex = 0;
+          for(GroupDocs<Integer> group : joinGroups.groups) {
+            JSONObject o3 = new JSONObject();
+            o2.add(o3);
+            if (fields != null || highlightFields != null) {
+              JSONObject o4 = new JSONObject();
+              o3.put("fields", o4);
+              // nocommit where does parent score come
+              // from ...
+              ScoreDoc sd = new ScoreDoc(group.groupValue.intValue(), 0.0f);
+              fillFields(state, highlighter, s.searcher, o4, sd, fields, highlights, hitIndex, sort);
+            }
+            hitIndex++;
+
+            o3.put("totalHits", group.totalHits);
+
+            if (!Float.isNaN(group.maxScore)) {
+              o3.put("maxScore", group.maxScore);
+            }
+
+            JSONObject o4 = new JSONObject();
+            o3.put("groupSortFields", o4);
+            SortField[] groupSortFields = (child.sort == null ? Sort.RELEVANCE : child.sort).getSort();
+            for(int i=0;i<groupSortFields.length;i++) {
+              String field = groupSortFields[i].getField();
+              if (field == null) {
+                field = "<score>";
               }
+              o4.put(field, group.groupSortValues[i]);
             }
 
-          } else {
-            o.put("totalHits", hits.totalHits);
-            JSONArray o2 = new JSONArray();
-            o.put("hits", o2);
-            if (!Float.isNaN(hits.getMaxScore())) {
-              o.put("maxScore", hits.getMaxScore());
-            }
+            JSONArray o5 = new JSONArray();
+            o3.put("hits", o5);
 
-            for(int hitIndex=0;hitIndex<hits.scoreDocs.length;hitIndex++) {
-              ScoreDoc hit = hits.scoreDocs[hitIndex];
-
-              JSONObject o3 = new JSONObject();
-              o2.add(o3);
-              o3.put("doc", hit.doc);
+            for(ScoreDoc hit : group.scoreDocs) {
+              JSONObject o6 = new JSONObject();
+              o5.add(o6);
+              o6.put("doc", hit.doc);
               if (!Float.isNaN(hit.score)) {
-                o3.put("score", hit.score);
+                o6.put("score", hit.score);
               }
 
               if (fields != null || highlightFields != null) {
-                JSONObject o4 = new JSONObject();
-                o3.put("fields", o4);
-                fillFields(state, highlighter, s.searcher, o4, hit, fields, highlights, hitIndex, sort);
+                JSONObject o7 = new JSONObject();
+                o6.put("fields", o7);
+                fillFields(state, highlighter, s.searcher, o7, hit, fields, highlights, hitIndex, child.sort);
               }
+
+              hitIndex++;
             }
           }
+        }
 
-          JSONObject o3 = new JSONObject();
-          o.put("searchState", o3);
-          o3.put("timeStamp", timeStamp);
+      } else {
+        result.put("totalHits", hits.totalHits);
+        JSONArray o2 = new JSONArray();
+        result.put("hits", o2);
+        if (!Float.isNaN(hits.getMaxScore())) {
+          result.put("maxScore", hits.getMaxScore());
+        }
 
-          o3.put("searcher", searcherToken);
-          if (hits != null && hits.scoreDocs.length != 0) {
-            ScoreDoc lastHit = hits.scoreDocs[hits.scoreDocs.length-1];
-            o3.put("lastDoc", lastHit.doc);
-            if (sort != null) {
-              JSONArray fieldValues = new JSONArray();
-              o3.put("lastFieldValues", fieldValues);
-              FieldDoc fd = (FieldDoc) lastHit;
-              for(Object fv : fd.fields) {
-                fieldValues.add(fv);
-              }
-            } else {
-              o3.put("lastScore", lastHit.score);
-            }
+        for(int hitIndex=0;hitIndex<hits.scoreDocs.length;hitIndex++) {
+          ScoreDoc hit = hits.scoreDocs[hitIndex];
+
+          JSONObject o3 = new JSONObject();
+          o2.add(o3);
+          o3.put("doc", hit.doc);
+          if (!Float.isNaN(hit.score)) {
+            o3.put("score", hit.score);
           }
 
-          if (fsp != null) {
-            JSONArray o5 = new JSONArray();
-            o.put("facets", o5);
-            Set<Integer> seenOrds = new HashSet<Integer>();
+          if (fields != null || highlightFields != null) {
+            JSONObject o4 = new JSONObject();
+            o3.put("fields", o4);
+            fillFields(state, highlighter, s.searcher, o4, hit, fields, highlights, hitIndex, sort);
+          }
+        }
+      }
 
-            // So we can lookup FacetResult keyed by its
-            // corresponding FacetRequest:
-            Map<FacetRequest,FacetResult> byRequest = new HashMap<FacetRequest,FacetResult>();
-            if (facets != null) {
-              for(FacetResult fr : facets) {
-                byRequest.put(fr.getFacetRequest(), fr);
-              }
-            }
+      JSONObject o3 = new JSONObject();
+      result.put("searchState", o3);
+      o3.put("timeStamp", timeStamp);
 
-            // Collate facet results, from either the cached
-            // results or computed results:
-            for(FacetRequest facetRequest : fsp.facetRequests) {
-              FacetResult fr = byRequest.get(facetRequest);
-              if (fr == null) {
-                fr = cachedFacetResults.get(facetRequest);
-                assert fr != null;
-              }
-              JSONObject fo = new JSONObject();
-              o5.add(fo);
-              JSONArray o6 = new JSONArray();
-              fo.put("counts", o6);
-              fo.put("numValidDescendants", fr.getNumValidDescendants());
-              FacetResultNode n = fr.getFacetResultNode();
-              JSONArray pair = new JSONArray();
-              o6.add(pair);
-              pair.add("top");
-              pair.add((int) n.value);
-              int childIDX = fr.getFacetRequest().categoryPath.components.length;
-              for(FacetResultNode childNode : n.subResults) {
-                pair = new JSONArray();
-                o6.add(pair);
-                pair.add(childNode.label.components[childIDX]);
-                pair.add((int) childNode.value);
-                seenOrds.add(childNode.ordinal);
-              }
-            }
+      // Record searcher version that handled this request:
+      o3.put("searcher", ((DirectoryReader) s.searcher.getIndexReader()).getVersion());
 
-            // Separately add facet count for any drill
-            // downs, if those values were not already
-            // included in the facet requests.  For example,
-            // this can happen in the books UI if the user
-            // drills down on the author of a particular
-            // book but that author doesn't make the top 10:
-            JSONArray extra = null;
-            for(CategoryPath cp : allDrillDowns) {
-              if (!facetRequestDims.contains(cp.components[0])) {
-                continue;
-              }
-              int ord = s.taxonomyReader.getOrdinal(cp);
-              if (ord == -1) {
-                r.fail("drillDowns", "path " + cp + " does not exist");
-              } else if (!seenOrds.contains(ord)) {
-                if (extra == null) {
-                  extra = new JSONArray();
-                  o.put("drillDownFacets", extra);
-                }
-                FacetArrays array = dimFacetArrays.get(cp.components[0]);
-                if (array == null) {
-                  array = dimFacetArrays.get(null);
-                }
-                assert array != null;
-                JSONArray facet = new JSONArray();
-                extra.add(facet);
-                JSONArray path = new JSONArray();
-                facet.add(path);
-                for(int i=0;i<cp.length;i++) {
-                  path.add(cp.components[i]);
-                }
-                facet.add(array.getIntArray()[ord]);
-              }
-            }
+      // Fill in lastDoc for searchAfter:
+      if (hits != null && hits.scoreDocs.length != 0) {
+        ScoreDoc lastHit = hits.scoreDocs[hits.scoreDocs.length-1];
+        o3.put("lastDoc", lastHit.doc);
+        if (sort != null) {
+          JSONArray fieldValues = new JSONArray();
+          o3.put("lastFieldValues", fieldValues);
+          FieldDoc fd = (FieldDoc) lastHit;
+          for(Object fv : fd.fields) {
+            fieldValues.add(fv);
           }
+        } else {
+          o3.put("lastScore", lastHit.score);
+        }
+      }
 
-          diagnostics.put("getFieldsMS", ((System.nanoTime()-t0)/1000000));
+      diagnostics.put("getFieldsMS", ((System.nanoTime()-t0)/1000000));
 
-          t0 = System.nanoTime();
-          String ret = o.toString();
-          //System.out.println("MS: " + ((System.nanoTime()-t0)/1000000.0));
-          return ret;
+      t0 = System.nanoTime();
+      resultString = result.toString();
+      //System.out.println("MS: " + ((System.nanoTime()-t0)/1000000.0));
         
-        } finally {
-          // NOTE: this is a little iffy, because we may not
-          // have obtained this searcher from the NRTManager
-          // (i.e. sometimes we pulled from
-          // SearcherLifetimeManager, other times (if
-          // snapshot was specified) we opened ourselves,
-          // but under-the-hood all this method does is
-          // s.getIndexReader().decRef(), which "works" for
-          // all ways:
-          state.manager.release(s);
-        }
+    } finally {
+      // NOTE: this is a little iffy, because we may not
+      // have obtained this searcher from the NRTManager
+      // (i.e. sometimes we pulled from
+      // SearcherLifetimeManager, other times (if
+      // snapshot was specified) we opened ourselves,
+      // but under-the-hood all these methods just call
+      // s.getIndexReader().decRef(), which is what release
+      // does:
+      state.manager.release(s);
+    }
+
+    return new FinishRequest() {
+      @Override
+      public String finish() throws IOException {
+        return resultString;
       }
     };
   }

Modified: lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/StartIndexHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/StartIndexHandler.java?rev=1555629&r1=1555628&r2=1555629&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/StartIndexHandler.java (original)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/StartIndexHandler.java Sun Jan  5 21:04:58 2014
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.lucene.facet.search.SearcherTaxonomyManager.SearcherAndTaxonomy;
+import org.apache.lucene.facet.taxonomy.SearcherTaxonomyManager.SearcherAndTaxonomy;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.server.FinishRequest;
 import org.apache.lucene.server.GlobalState;

Modified: lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/StatsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/StatsHandler.java?rev=1555629&r1=1555628&r2=1555629&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/StatsHandler.java (original)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/handlers/StatsHandler.java Sun Jan  5 21:04:58 2014
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.lucene.facet.search.SearcherTaxonomyManager.SearcherAndTaxonomy;
+import org.apache.lucene.facet.taxonomy.SearcherTaxonomyManager.SearcherAndTaxonomy;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.search.CachingWrapperFilter;
 import org.apache.lucene.search.IndexSearcher;
@@ -81,7 +81,6 @@ public class StatsHandler extends Handle
               searchers.put(Long.toString(((DirectoryReader) searcher.getIndexReader()).getVersion()), s);
               s.put("staleAgeSeconds", ageSec);
               s.put("segments", searcher.getIndexReader().toString());
-              s.put("facetCacheSizeBytes", RamUsageEstimator.sizeOf(((MyIndexSearcher) searcher).topFacetsCache));
               return false;
             }
           });

Modified: lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/params/Request.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/params/Request.java?rev=1555629&r1=1555628&r2=1555629&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/params/Request.java (original)
+++ lucene/dev/branches/lucene5376/lucene/server/src/java/org/apache/lucene/server/params/Request.java Sun Jan  5 21:04:58 2014
@@ -278,14 +278,18 @@ public class Request {
         return null;
       }
     } else {
-      if (!(v instanceof String)) {
-        fail(name, "expected String but got " + v.getClass());
+      if ((v instanceof String) == false) {
+        fail(name, "expected String but got " + toSimpleString(v.getClass()));
       }
       params.remove(name);
       return (String) v;
     }
   }
 
+  private static String toSimpleString(Class<?> cl) {
+    return cl.getSimpleName();
+  }
+
   /** Retrieve an enum parameter.  Once this is called
    *  for a given parameter it cannot be called again on
    *  that parameter. */