You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by yo...@apache.org on 2015/07/10 05:47:54 UTC

svn commit: r1690199 - in /lucene/dev/branches/branch_5x: ./ solr/ solr/core/ solr/core/src/java/org/apache/solr/search/facet/

Author: yonik
Date: Fri Jul 10 03:47:54 2015
New Revision: 1690199

URL: http://svn.apache.org/r1690199
Log:
SOLR-7455: defer non-sorting facet stats

Added:
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java
      - copied, changed from r1690189, lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java
Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/solr/   (props changed)
    lucene/dev/branches/branch_5x/solr/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/solr/core/   (props changed)
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetField.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorNumeric.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/UnInvertedField.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/UniqueSlotAcc.java

Modified: lucene/dev/branches/branch_5x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/CHANGES.txt?rev=1690199&r1=1690198&r2=1690199&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/solr/CHANGES.txt Fri Jul 10 03:47:54 2015
@@ -173,6 +173,16 @@ Optimizations
 * SOLR-7751: Minor optimizations to QueryComponent.process (reduce eager instantiations,
   cache method calls) (Christine Poerschke via Ramkumar Aiyengar)
 
+* SOLR-7455: Terms facets with the JSON Facet API now defer calculating non-sorting stats
+  until a second phase, after the top N facets are found. This improves performance
+  proportional to the number of non-sorting statistics being calculated in addition to
+  the number of buckets and domain documents.
+  For Example: The facet request  {type:terms, field:field1, facet:{x:"unique(field2)"}}
+  saw a 7x improvement when field1 and 1M unique terms and field2 had 1000 unique terms.
+  (yonik)
+  
+
+
 Other Changes
 ----------------------
 

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetField.java?rev=1690199&r1=1690198&r2=1690199&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetField.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetField.java Fri Jul 10 03:47:54 2015
@@ -20,8 +20,11 @@ package org.apache.solr.search.facet;
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.Iterator;
+import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.Fields;
@@ -37,6 +40,7 @@ import org.apache.lucene.index.TermsEnum
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -62,12 +66,12 @@ public class FacetField extends FacetReq
   long limit = 10;
   long mincount = 1;
   boolean missing;
+  boolean allBuckets;   // show cumulative stats across all buckets (this can be different than non-bucketed stats across all docs because of multi-valued docs)
   boolean numBuckets;
   String prefix;
   String sortVariable;
   SortDirection sortDirection;
   FacetMethod method;
-  boolean allBuckets;   // show cumulative stats across all buckets (this can be different than non-bucketed stats across all docs because of multi-valued docs)
   int cacheDf;  // 0 means "default", -1 means "never cache"
 
   // TODO: put this somewhere more generic?
@@ -134,6 +138,7 @@ public class FacetField extends FacetReq
         return new FacetFieldProcessorNumeric(fcontext, this, sf);
       } else {
         // single valued string...
+//        return new FacetFieldProcessorDV(fcontext, this, sf);
         return new FacetFieldProcessorDV(fcontext, this, sf);
         // what about FacetFieldProcessorFC?
       }
@@ -153,10 +158,25 @@ public class FacetField extends FacetReq
 
 abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
   SchemaField sf;
-  SlotAcc sortAcc;
   SlotAcc indexOrderAcc;
   int effectiveMincount;
 
+  Map<String,AggValueSource> deferredAggs;  // null if none
+
+  // TODO: push any of this down to base class?
+
+  //
+  // For sort="x desc", collectAcc would point to "x", and sortAcc would also point to "x".
+  // collectAcc would be used to accumulate all buckets, and sortAcc would be used to sort those buckets.
+  //
+  SlotAcc collectAcc;  // Accumulator to collect across entire domain (in addition to the countAcc).  May be null.
+  SlotAcc sortAcc;     // Accumulator to use for sorting *only* (i.e. not used for collection). May be an alias of countAcc, collectAcc, or indexOrderAcc
+  SlotAcc[] otherAccs; // Accumulators that do not need to be calculated across all buckets.
+
+  SpecialSlotAcc allBucketsAcc;  // this can internally refer to otherAccs and/or collectAcc. setNextReader should be called on otherAccs directly if they exist.
+  SpecialSlotAcc missingAcc;     // this can internally refer to otherAccs and/or collectAcc. setNextReader should be called on otherAccs directly if they exist.
+
+
   FacetFieldProcessor(FacetContext fcontext, FacetField freq, SchemaField sf) {
     super(fcontext, freq);
     this.sf = sf;
@@ -168,21 +188,221 @@ abstract class FacetFieldProcessor exten
     return response;
   }
 
-  void setSortAcc(int numSlots) {
-    if (indexOrderAcc == null) {
-      // This sorting accumulator just goes by the slot number, so does not need to be collected
-      // and hence does not need to find it's way into the accMap or accs array.
-      indexOrderAcc = new SortSlotAcc(fcontext);
+  // This is used to create accs for second phase (or to create accs for all aggs)
+  @Override
+  protected void createAccs(int docCount, int slotCount) throws IOException {
+    if (accMap == null) {
+      accMap = new LinkedHashMap<>();
+    }
+
+    // allow a custom count acc to be used
+    if (countAcc == null) {
+      countAcc = new CountSlotArrAcc(fcontext, slotCount);
+      countAcc.key = "count";
+    }
+
+    if (accs != null) {
+      // reuse these accs, but reset them first
+      for (SlotAcc acc : accs) {
+        acc.reset();
+      }
+      return;
+    } else {
+      accs = new SlotAcc[ freq.getFacetStats().size() ];
+    }
+
+    int accIdx = 0;
+    for (Map.Entry<String,AggValueSource> entry : freq.getFacetStats().entrySet()) {
+      SlotAcc acc = null;
+      if (slotCount == 1) {
+        acc = accMap.get(entry.getKey());
+        if (acc != null) {
+          acc.reset();
+        }
+      }
+      if (acc == null) {
+        acc = entry.getValue().createSlotAcc(fcontext, docCount, slotCount);
+        acc.key = entry.getKey();
+        accMap.put(acc.key, acc);
+      }
+      accs[accIdx++] = acc;
+    }
+  }
+
+  void createCollectAcc(int numDocs, int numSlots) throws IOException {
+    accMap = new LinkedHashMap<>();
+
+    // we always count...
+    // allow a subclass to set a custom counter.
+    if (countAcc == null) {
+      countAcc = new CountSlotArrAcc(fcontext, numSlots);
+    }
+
+    if ("count".equals(freq.sortVariable)) {
+      sortAcc = countAcc;
+      deferredAggs = freq.getFacetStats();
+    } else if ("index".equals(freq.sortVariable)) {
+      // allow subclass to set indexOrderAcc first
+      if (indexOrderAcc == null) {
+        // This sorting accumulator just goes by the slot number, so does not need to be collected
+        // and hence does not need to find it's way into the accMap or accs array.
+        indexOrderAcc = new SortSlotAcc(fcontext);
+      }
+      sortAcc = indexOrderAcc;
+      deferredAggs = freq.getFacetStats();
+    } else {
+      AggValueSource sortAgg = freq.getFacetStats().get(freq.sortVariable);
+      if (sortAgg != null) {
+        collectAcc = sortAgg.createSlotAcc(fcontext, numDocs, numSlots);
+        collectAcc.key = freq.sortVariable; // TODO: improve this
+      }
+      sortAcc = collectAcc;
+      deferredAggs = new HashMap<>(freq.getFacetStats());
+      deferredAggs.remove(freq.sortVariable);
     }
 
-    String sortKey = freq.sortVariable;
-    sortAcc = accMap.get(sortKey);
-
-    if (sortAcc == null) {
-      if ("count".equals(sortKey)) {
-        sortAcc = countAcc;
-      } else if ("index".equals(sortKey)) {
-        sortAcc = indexOrderAcc;
+    if (deferredAggs.size() == 0) {
+      deferredAggs = null;
+    }
+
+    boolean needOtherAccs = freq.allBuckets;  // TODO: use for missing too...
+
+    if (!needOtherAccs) {
+      // we may need them later, but we don't want to create them now
+      // otherwise we won't know if we need to call setNextReader on them.
+      return;
+    }
+
+    // create the deffered aggs up front for use by allBuckets
+    createOtherAccs(numDocs, 1);
+  }
+
+
+  void createOtherAccs(int numDocs, int numSlots) throws IOException {
+    if (otherAccs != null) {
+      // reuse existing accumulators
+      for (SlotAcc acc : otherAccs) {
+        acc.reset();  // todo - make reset take numDocs and numSlots?
+      }
+      return;
+    }
+
+    int numDeferred = deferredAggs == null ? 0 : deferredAggs.size();
+    if (numDeferred <= 0) return;
+
+    otherAccs = new SlotAcc[ numDeferred ];
+
+    int otherAccIdx = 0;
+    for (Map.Entry<String,AggValueSource> entry : deferredAggs.entrySet()) {
+      AggValueSource agg = entry.getValue();
+      SlotAcc acc = agg.createSlotAcc(fcontext, numDocs, numSlots);
+      acc.key = entry.getKey();
+      accMap.put(acc.key, acc);
+      otherAccs[otherAccIdx++] = acc;
+    }
+
+    if (numDeferred == freq.getFacetStats().size()) {
+      // accs and otherAccs are the same...
+      accs = otherAccs;
+    }
+  }
+
+
+  int collectFirstPhase(DocSet docs, int slot) throws IOException {
+    int num = -1;
+    if (collectAcc != null) {
+      num = collectAcc.collect(docs, slot);
+    }
+    if (allBucketsAcc != null) {
+      num = allBucketsAcc.collect(docs, slot);
+    }
+    return num >= 0 ? num : docs.size();
+  }
+
+  void collectFirstPhase(int segDoc, int slot) throws IOException {
+    if (collectAcc != null) {
+      collectAcc.collect(segDoc, slot);
+    }
+    if (allBucketsAcc != null) {
+      allBucketsAcc.collect(segDoc, slot);
+    }
+  }
+
+
+  void fillBucket(SimpleOrderedMap<Object> target, int count, int slotNum, DocSet subDomain, Query filter) throws IOException {
+    target.add("count", count);
+    if (count <= 0 && !freq.processEmpty) return;
+
+    if (collectAcc != null && slotNum >= 0) {
+      collectAcc.setValues(target, slotNum);
+    }
+
+    createOtherAccs(-1, 1);
+
+    if (otherAccs == null && freq.subFacets.isEmpty()) return;
+
+    if (subDomain == null) {
+      subDomain = fcontext.searcher.getDocSet(filter, fcontext.base);
+    }
+
+    // if no subFacets, we only need a DocSet
+    // otherwise we need more?
+    // TODO: save something generic like "slotNum" in the context and use that to implement things like filter exclusion if necessary?
+    // Hmmm, but we need to look up some stuff anyway (for the label?)
+    // have a method like "DocSet applyConstraint(facet context, DocSet parent)"
+    // that's needed for domain changing things like joins anyway???
+
+    if (otherAccs != null) {
+      // do acc at a time (traversing domain each time) or do all accs for each doc?
+      for (SlotAcc acc : otherAccs) {
+        acc.reset(); // TODO: only needed if we previously used for allBuckets or missing
+        acc.collect(subDomain, 0);
+        acc.setValues(target, 0);
+      }
+    }
+
+    processSubs(target, filter, subDomain);
+  }
+
+
+  @Override
+  protected void processStats(SimpleOrderedMap<Object> bucket, DocSet docs, int docCount) throws IOException {
+    if (docCount == 0 && !freq.processEmpty || freq.getFacetStats().size() == 0) {
+      bucket.add("count", docCount);
+      return;
+    }
+    createAccs(docCount, 1);
+    int collected = collect(docs, 0);
+
+    // countAcc.incrementCount(0, collected);  // should we set the counton the acc instead of just passing it?
+
+    assert collected == docCount;
+    addStats(bucket, collected, 0);
+  }
+
+  // overrides but with different signature!
+  void addStats(SimpleOrderedMap<Object> target, int count, int slotNum) throws IOException {
+    target.add("count", count);
+    if (count > 0 || freq.processEmpty) {
+      for (SlotAcc acc : accs) {
+        acc.setValues(target, slotNum);
+      }
+    }
+  }
+
+  @Override
+  void setNextReader(LeafReaderContext ctx) throws IOException {
+    // base class calls this (for missing bucket...) ...  go over accs[] in that case
+    super.setNextReader(ctx);
+  }
+
+  void setNextReaderFirstPhase(LeafReaderContext ctx) throws IOException {
+    if (collectAcc != null) {
+      collectAcc.setNextReader(ctx);
+    }
+    if (otherAccs != null) {
+      for (SlotAcc acc : otherAccs) {
+        acc.setNextReader(ctx);
       }
     }
   }
@@ -195,6 +415,82 @@ abstract class FacetFieldProcessor exten
   }
 }
 
+class SpecialSlotAcc extends SlotAcc {
+  SlotAcc collectAcc;
+  SlotAcc[] otherAccs;
+  int collectAccSlot;
+  int otherAccsSlot;
+  long count;
+
+  public SpecialSlotAcc(FacetContext fcontext, SlotAcc collectAcc, int collectAccSlot, SlotAcc[] otherAccs, int otherAccsSlot) {
+    super(fcontext);
+    this.collectAcc = collectAcc;
+    this.collectAccSlot = collectAccSlot;
+    this.otherAccs = otherAccs;
+    this.otherAccsSlot = otherAccsSlot;
+  }
+
+  public int getCollectAccSlot() { return collectAccSlot; }
+  public int getOtherAccSlot() { return otherAccsSlot; }
+
+  public long getSpecialCount() {
+    return count;
+  }
+
+  @Override
+  public void collect(int doc, int slot) throws IOException {
+    assert slot != collectAccSlot || slot < 0;
+    count++;
+    if (collectAcc != null) {
+      collectAcc.collect(doc, collectAccSlot);
+    }
+    if (otherAccs != null) {
+      for (SlotAcc otherAcc : otherAccs) {
+        otherAcc.collect(doc, otherAccsSlot);
+      }
+    }
+  }
+
+  @Override
+  public int compare(int slotA, int slotB) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Object getValue(int slotNum) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setValues(SimpleOrderedMap<Object> bucket, int slotNum) throws IOException {
+    if (collectAcc != null) {
+      collectAcc.setValues(bucket, collectAccSlot);
+    }
+    if (otherAccs != null) {
+      for (SlotAcc otherAcc : otherAccs) {
+        otherAcc.setValues(bucket, otherAccsSlot);
+      }
+    }
+  }
+
+  @Override
+  public void reset() {
+    // reset should be called on underlying accs
+    // TODO: but in case something does need to be done here, should we require this method to be called but do nothing for now?
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void resize(Resizer resizer) {
+    // someone else will call resize on collectAcc directly
+    if (collectAccSlot >= 0) {
+      collectAccSlot = resizer.getNewSlot(collectAccSlot);
+    }
+  }
+}
+
+
+
 
 // base class for FC style of facet counting (single and multi-valued strings)
 abstract class FacetFieldProcessorFCBase extends FacetFieldProcessor {
@@ -204,8 +500,9 @@ abstract class FacetFieldProcessorFCBase
   int nTerms;
   int nDocs;
   int maxSlots;
-  int allBucketsSlot;
 
+  int allBucketsSlot = -1;  // slot for the primary Accs (countAcc, collectAcc)
+  int missingSlot = -1;
 
   public FacetFieldProcessorFCBase(FacetContext fcontext, FacetField freq, SchemaField sf) {
     super(fcontext, freq, sf);
@@ -235,19 +532,25 @@ abstract class FacetFieldProcessorFCBase
 
     findStartAndEndOrds();
 
-    // if we need an extra slot for the "missing" bucket, and it wasn't able to be tacked onto the beginning,
-    // then lets add room for it at the end.
-    maxSlots = (freq.missing && startTermIndex != -1) ? nTerms + 1 : nTerms;
+    maxSlots = nTerms;
 
     if (freq.allBuckets) {
-      allBucketsSlot = maxSlots;
-      maxSlots++;
-    } else {
-      allBucketsSlot = -1;
+      allBucketsSlot = maxSlots++;
+    }
+    if (freq.missing) {
+      missingSlot = maxSlots++;
+    }
+
+    createCollectAcc(nDocs, maxSlots);
+
+    if (freq.allBuckets) {
+      allBucketsAcc = new SpecialSlotAcc(fcontext, collectAcc, allBucketsSlot, otherAccs, 0);
+    }
+
+    if (freq.missing) {
+      // TODO: optimize case when missingSlot can be contiguous with other slots
+      missingAcc = new SpecialSlotAcc(fcontext, collectAcc, missingSlot, otherAccs, 1);
     }
-    createAccs(nDocs, maxSlots);
-    setSortAcc(maxSlots);
-    prepareForCollection();
 
     collectDocs();
 
@@ -284,7 +587,8 @@ abstract class FacetFieldProcessorFCBase
 
     Slot bottom = null;
     for (int i = (startTermIndex == -1) ? 1 : 0; i < nTerms; i++) {
-      if (countAcc.getCount(i) < effectiveMincount) {
+      // screen out buckets not matching mincount immediately (i.e. don't even increment numBuckets)
+      if (effectiveMincount > 0 && countAcc.getCount(i) < effectiveMincount) {
         continue;
       }
 
@@ -334,9 +638,9 @@ abstract class FacetFieldProcessorFCBase
 
     if (freq.allBuckets) {
       SimpleOrderedMap<Object> allBuckets = new SimpleOrderedMap<>();
-      countAcc.setValues(allBuckets, allBucketsSlot);
-      for (SlotAcc acc : accs) {
-        acc.setValues(allBuckets, allBucketsSlot);
+      allBuckets.add("count", allBucketsAcc.getSpecialCount());
+      if (allBucketsAcc != null) {
+        allBucketsAcc.setValues(allBuckets, allBucketsSlot);
       }
       res.add("allBuckets", allBuckets);
     }
@@ -345,6 +649,9 @@ abstract class FacetFieldProcessorFCBase
     res.add("buckets", bucketList);
 
 
+    // TODO: do this with a callback instead?
+    boolean needFilter = deferredAggs != null || freq.getSubFacets().size() > 0;
+
     for (int slotNum : sortedSlots) {
       SimpleOrderedMap<Object> bucket = new SimpleOrderedMap<>();
 
@@ -355,52 +662,38 @@ abstract class FacetFieldProcessorFCBase
       Object val = sf.getType().toObject(sf, br);
 
       bucket.add("val", val);
-      // add stats for this bucket
-      addStats(bucket, slotNum);
 
-      // handle sub-facets for this bucket
-      if (freq.getSubFacets().size() > 0) {
-        TermQuery filter = new TermQuery(new Term(sf.getName(), br.clone()));
-        try {
-          processSubs(bucket, filter, fcontext.searcher.getDocSet(filter, fcontext.base) );
-        } finally {
-          // subContext.base.decref();  // OFF-HEAP
-          // subContext.base = null;  // do not modify context after creation... there may be deferred execution (i.e. streaming)
-        }
-      }
+      TermQuery filter = needFilter ? new TermQuery(new Term(sf.getName(), br.clone())) : null;
+      fillBucket(bucket, countAcc.getCount(slotNum), slotNum, null, filter);
 
       bucketList.add(bucket);
     }
 
     if (freq.missing) {
       SimpleOrderedMap<Object> missingBucket = new SimpleOrderedMap<>();
-      DocSet missingDocSet = null;
-      try {
-        if (startTermIndex == -1) {
-          addStats(missingBucket, 0);
-        } else {
-          missingDocSet = getFieldMissing(fcontext.searcher, fcontext.base, freq.field);
-          // an extra slot was added to the end for this missing bucket
-          countAcc.incrementCount(nTerms, missingDocSet.size());
-          collect(missingDocSet, nTerms);
-          addStats(missingBucket, nTerms);
-        }
+      fillBucket(missingBucket, getFieldMissingQuery(fcontext.searcher, freq.field));
+      res.add("missing", missingBucket);
 
-        if (freq.getSubFacets().size() > 0) {
-          // TODO: we can do better than this!
-          if (missingDocSet == null) {
-            missingDocSet = getFieldMissing(fcontext.searcher, fcontext.base, freq.field);
-          }
-          processSubs(missingBucket, getFieldMissingQuery(fcontext.searcher, freq.field), missingDocSet);
-        }
+      /*** TODO - OPTIMIZE
+      DocSet missingDocSet = null;
+      if (startTermIndex == -1) {
+        fillBucket(missingBucket, countAcc.getCount(0), null);
+      } else {
+        missingDocSet = getFieldMissing(fcontext.searcher, fcontext.base, freq.field);
+        // an extra slot was added to the end for this missing bucket
+        countAcc.incrementCount(nTerms, missingDocSet.size());
+        collect(missingDocSet, nTerms);
+        addStats(missingBucket, nTerms);
+      }
 
-        res.add("missing", missingBucket);
-      } finally {
-        if (missingDocSet != null) {
-          // missingDocSet.decref(); // OFF-HEAP
-          missingDocSet = null;
+      if (freq.getSubFacets().size() > 0) {
+        // TODO: we can do better than this!
+        if (missingDocSet == null) {
+          missingDocSet = getFieldMissing(fcontext.searcher, fcontext.base, freq.field);
         }
+        processSubs(missingBucket, getFieldMissingQuery(fcontext.searcher, freq.field), missingDocSet);
       }
+       ***/
     }
 
     return res;
@@ -410,31 +703,51 @@ abstract class FacetFieldProcessorFCBase
 }
 
 
-class FacetFieldProcessorFC extends FacetFieldProcessorFCBase {
-  SortedDocValues sortedDocValues;
+class FacetFieldProcessorDV extends FacetFieldProcessorFCBase {
+  static boolean unwrap_singleValued_multiDv = true;  // only set to false for test coverage
+
+  boolean multiValuedField;
+  SortedSetDocValues si;  // only used for term lookups (for both single and multi-valued)
+  MultiDocValues.OrdinalMap ordinalMap = null; // maps per-segment ords to global ords
 
 
-  public FacetFieldProcessorFC(FacetContext fcontext, FacetField freq, SchemaField sf) {
+  public FacetFieldProcessorDV(FacetContext fcontext, FacetField freq, SchemaField sf) {
     super(fcontext, freq, sf);
+    multiValuedField = sf.multiValued() || sf.getType().multiValuedFieldCache();
   }
 
   protected BytesRef lookupOrd(int ord) throws IOException {
-    return sortedDocValues.lookupOrd(ord);
+    return si.lookupOrd(ord);
   }
 
   protected void findStartAndEndOrds() throws IOException {
-    sortedDocValues = FieldUtil.getSortedDocValues(fcontext.qcontext, sf, null);
+    if (multiValuedField) {
+      si = FieldUtil.getSortedSetDocValues(fcontext.qcontext, sf, null);
+      if (si instanceof MultiDocValues.MultiSortedSetDocValues) {
+        ordinalMap = ((MultiDocValues.MultiSortedSetDocValues)si).mapping;
+      }
+    } else {
+      SortedDocValues single = FieldUtil.getSortedDocValues(fcontext.qcontext, sf, null);
+      si = DocValues.singleton(single);  // multi-valued view
+      if (single instanceof MultiDocValues.MultiSortedDocValues) {
+        ordinalMap = ((MultiDocValues.MultiSortedDocValues)single).mapping;
+      }
+    }
+
+    if (si.getValueCount() >= Integer.MAX_VALUE) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Field has too many unique values. field=" + sf + " nterms= " + si.getValueCount());
+    }
 
     if (prefixRef != null) {
-      startTermIndex = sortedDocValues.lookupTerm(prefixRef.get());
+      startTermIndex = (int)si.lookupTerm(prefixRef.get());
       if (startTermIndex < 0) startTermIndex = -startTermIndex - 1;
       prefixRef.append(UnicodeUtil.BIG_TERM);
-      endTermIndex = sortedDocValues.lookupTerm(prefixRef.get());
+      endTermIndex = (int)si.lookupTerm(prefixRef.get());
       assert endTermIndex < 0;
       endTermIndex = -endTermIndex - 1;
     } else {
       startTermIndex = 0;
-      endTermIndex = sortedDocValues.getValueCount();
+      endTermIndex = (int)si.getValueCount();
     }
 
     // optimize collecting the "missing" bucket when startTermindex is 0 (since the "missing" ord is -1)
@@ -443,43 +756,116 @@ class FacetFieldProcessorFC extends Face
     nTerms = endTermIndex - startTermIndex;
   }
 
+  @Override
   protected void collectDocs() throws IOException {
+    if (nTerms <= 0 || fcontext.base.size() < effectiveMincount) { // TODO: what about allBuckets? missing bucket?
+      return;
+    }
+
     final List<LeafReaderContext> leaves = fcontext.searcher.getIndexReader().leaves();
-    final Iterator<LeafReaderContext> ctxIt = leaves.iterator();
-    LeafReaderContext ctx = null;
-    int segBase = 0;
-    int segMax;
-    int adjustedMax = 0;
-    for (DocIterator docsIt = fcontext.base.iterator(); docsIt.hasNext(); ) {
-      final int doc = docsIt.nextDoc();
-      if (doc >= adjustedMax) {
-        do {
-          ctx = ctxIt.next();
-          segBase = ctx.docBase;
-          segMax = ctx.reader().maxDoc();
-          adjustedMax = segBase + segMax;
-        } while (doc >= adjustedMax);
-        assert doc >= ctx.docBase;
-        setNextReader(ctx);
-      }
-
-      int term = sortedDocValues.getOrd( doc );
-      int arrIdx = term - startTermIndex;
-      if (arrIdx>=0 && arrIdx<nTerms) {
-        countAcc.incrementCount(arrIdx, 1);
-        collect(doc - segBase, arrIdx);  // per-seg collectors
-        if (allBucketsSlot >= 0 && term >= 0) {
-          countAcc.incrementCount(allBucketsSlot, 1);
-          collect(doc - segBase, allBucketsSlot);  // per-seg collectors
+    Filter filter = fcontext.base.getTopFilter();
+
+    for (int subIdx = 0; subIdx < leaves.size(); subIdx++) {
+      LeafReaderContext subCtx = leaves.get(subIdx);
+
+      setNextReaderFirstPhase(subCtx);
+
+      DocIdSet dis = filter.getDocIdSet(subCtx, null); // solr docsets already exclude any deleted docs
+      DocIdSetIterator disi = dis.iterator();
+
+      SortedDocValues singleDv = null;
+      SortedSetDocValues multiDv = null;
+      if (multiValuedField) {
+        // TODO: get sub from multi?
+        multiDv = subCtx.reader().getSortedSetDocValues(sf.getName());
+        if (multiDv == null) {
+          multiDv = DocValues.emptySortedSet();
+        }
+        // some codecs may optimize SortedSet storage for single-valued fields
+        // this will be null if this is not a wrapped single valued docvalues.
+        if (unwrap_singleValued_multiDv) {
+          singleDv = DocValues.unwrapSingleton(multiDv);
+        }
+      } else {
+        singleDv = subCtx.reader().getSortedDocValues(sf.getName());
+        if (singleDv == null) {
+          singleDv = DocValues.emptySorted();
         }
       }
+
+      LongValues toGlobal = ordinalMap == null ? null : ordinalMap.getGlobalOrds(subIdx);
+
+      if (singleDv != null) {
+        collectDocs(singleDv, disi, toGlobal);
+      } else {
+        collectDocs(multiDv, disi, toGlobal);
+      }
+    }
+
+  }
+
+  protected void collectDocs(SortedDocValues singleDv, DocIdSetIterator disi, LongValues toGlobal) throws IOException {
+    int doc;
+    while ((doc = disi.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+      int segOrd = singleDv.getOrd(doc);
+      collect(doc, segOrd, toGlobal);
+    }
+  }
+
+  protected void collectDocs(SortedSetDocValues multiDv, DocIdSetIterator disi, LongValues toGlobal) throws IOException {
+    int doc;
+    while ((doc = disi.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+      multiDv.setDocument(doc);
+      int segOrd = (int)multiDv.nextOrd();
+      collect(doc, segOrd, toGlobal); // collect anything the first time (even -1 for missing)
+      if (segOrd < 0) continue;
+      for(;;) {
+        segOrd = (int)multiDv.nextOrd();
+        if (segOrd < 0) break;
+        collect(doc, segOrd, toGlobal);
+      }
+    }
+  }
+
+  private void collect(int doc, int segOrd, LongValues toGlobal) throws IOException {
+    int ord = (toGlobal != null && segOrd >= 0) ? (int)toGlobal.get(segOrd) : segOrd;
+
+    int arrIdx = ord - startTermIndex;
+    if (arrIdx >= 0 && arrIdx < nTerms) {
+      countAcc.incrementCount(arrIdx, 1);
+      if (collectAcc != null) {
+        collectAcc.collect(doc, arrIdx);
+      }
+      // since this can be called for missing, we need to ensure it's currently not.
+      if (allBucketsAcc != null && ord >= 0) {
+        allBucketsAcc.collect(doc, arrIdx);
+      }
     }
   }
 
 }
 
+
+///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
 // UnInvertedField implementation of field faceting
-class FacetFieldProcessorUIF extends FacetFieldProcessorFC {
+class FacetFieldProcessorUIF extends FacetFieldProcessorFCBase {
   UnInvertedField uif;
   TermsEnum te;
 
@@ -615,10 +1001,7 @@ class FacetFieldProcessorStream extends
     hasSubFacets = freq.subFacets.size() > 0;
     bucketsToSkip = freq.offset;
 
-
-
     createAccs(-1, 1);
-    prepareForCollection();
 
     // Minimum term docFreq in order to use the filterCache for that term.
     int defaultMinDf = Math.max(fcontext.searcher.maxDoc() >> 4, 3);  // (minimum of 3 is for test coverage purposes)
@@ -846,143 +1229,3 @@ class FacetFieldProcessorStream extends
 }
 
 
-
-class FacetFieldProcessorDV extends FacetFieldProcessorFCBase {
-  static boolean unwrap_singleValued_multiDv = true;  // only set to false for test coverage
-
-  boolean multiValuedField;
-  SortedSetDocValues si;  // only used for term lookups (for both single and multi-valued)
-  MultiDocValues.OrdinalMap ordinalMap = null; // maps per-segment ords to global ords
-
-
-  public FacetFieldProcessorDV(FacetContext fcontext, FacetField freq, SchemaField sf) {
-    super(fcontext, freq, sf);
-    multiValuedField = sf.multiValued() || sf.getType().multiValuedFieldCache();
-  }
-
-  protected BytesRef lookupOrd(int ord) throws IOException {
-    return si.lookupOrd(ord);
-  }
-
-  protected void findStartAndEndOrds() throws IOException {
-    if (multiValuedField) {
-      si = FieldUtil.getSortedSetDocValues(fcontext.qcontext, sf, null);
-      if (si instanceof MultiDocValues.MultiSortedSetDocValues) {
-        ordinalMap = ((MultiDocValues.MultiSortedSetDocValues)si).mapping;
-      }
-    } else {
-      SortedDocValues single = FieldUtil.getSortedDocValues(fcontext.qcontext, sf, null);
-      si = DocValues.singleton(single);  // multi-valued view
-      if (single instanceof MultiDocValues.MultiSortedDocValues) {
-        ordinalMap = ((MultiDocValues.MultiSortedDocValues)single).mapping;
-      }
-    }
-
-    if (si.getValueCount() >= Integer.MAX_VALUE) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Field has too many unique values. field=" + sf + " nterms= " + si.getValueCount());
-    }
-
-    if (prefixRef != null) {
-      startTermIndex = (int)si.lookupTerm(prefixRef.get());
-      if (startTermIndex < 0) startTermIndex = -startTermIndex - 1;
-      prefixRef.append(UnicodeUtil.BIG_TERM);
-      endTermIndex = (int)si.lookupTerm(prefixRef.get());
-      assert endTermIndex < 0;
-      endTermIndex = -endTermIndex - 1;
-    } else {
-      startTermIndex = 0;
-      endTermIndex = (int)si.getValueCount();
-    }
-
-    // optimize collecting the "missing" bucket when startTermindex is 0 (since the "missing" ord is -1)
-    startTermIndex = startTermIndex==0 && freq.missing ? -1 : startTermIndex;
-
-    nTerms = endTermIndex - startTermIndex;
-  }
-
-  @Override
-  protected void collectDocs() throws IOException {
-     if (nTerms <= 0 || fcontext.base.size() < effectiveMincount) { // TODO: what about allBuckets? missing bucket?
-       return;
-     }
-
-    final List<LeafReaderContext> leaves = fcontext.searcher.getIndexReader().leaves();
-    Filter filter = fcontext.base.getTopFilter();
-
-    for (int subIdx = 0; subIdx < leaves.size(); subIdx++) {
-      LeafReaderContext subCtx = leaves.get(subIdx);
-
-      setNextReader(subCtx);
-
-      DocIdSet dis = filter.getDocIdSet(subCtx, null); // solr docsets already exclude any deleted docs
-      DocIdSetIterator disi = dis.iterator();
-
-      SortedDocValues singleDv = null;
-      SortedSetDocValues multiDv = null;
-      if (multiValuedField) {
-        // TODO: get sub from multi?
-        multiDv = subCtx.reader().getSortedSetDocValues(sf.getName());
-        if (multiDv == null) {
-          multiDv = DocValues.emptySortedSet();
-        }
-        // some codecs may optimize SortedSet storage for single-valued fields
-        // this will be null if this is not a wrapped single valued docvalues.
-        if (unwrap_singleValued_multiDv) {
-          singleDv = DocValues.unwrapSingleton(multiDv);
-        }
-      } else {
-        singleDv = subCtx.reader().getSortedDocValues(sf.getName());
-        if (singleDv == null) {
-          singleDv = DocValues.emptySorted();
-        }
-      }
-
-      LongValues toGlobal = ordinalMap == null ? null : ordinalMap.getGlobalOrds(subIdx);
-
-      if (singleDv != null) {
-        collectDocs(singleDv, disi, toGlobal);
-      } else {
-        collectDocs(multiDv, disi, toGlobal);
-      }
-    }
-
-  }
-
-  protected void collectDocs(SortedDocValues singleDv, DocIdSetIterator disi, LongValues toGlobal) throws IOException {
-    int doc;
-    while ((doc = disi.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-      int segOrd = singleDv.getOrd(doc);
-      collect(doc, segOrd, toGlobal);
-    }
-  }
-
-  protected void collectDocs(SortedSetDocValues multiDv, DocIdSetIterator disi, LongValues toGlobal) throws IOException {
-    int doc;
-    while ((doc = disi.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-      multiDv.setDocument(doc);
-      int segOrd = (int)multiDv.nextOrd();
-      collect(doc, segOrd, toGlobal); // collect anything the first time (even -1 for missing)
-      if (segOrd < 0) continue;
-      for(;;) {
-        segOrd = (int)multiDv.nextOrd();
-        if (segOrd < 0) break;
-        collect(doc, segOrd, toGlobal);
-      }
-    }
-  }
-
-  private void collect(int doc, int segOrd, LongValues toGlobal) throws IOException {
-    int ord = (toGlobal != null && segOrd >= 0) ? (int)toGlobal.get(segOrd) : segOrd;
-
-    int arrIdx = ord - startTermIndex;
-    if (arrIdx >= 0 && arrIdx < nTerms) {
-      countAcc.incrementCount(arrIdx, 1);
-      collect(doc, arrIdx);  // per-seg collectors
-      if (allBucketsSlot >= 0 && ord >= 0) {
-        countAcc.incrementCount(allBucketsSlot, 1);
-        collect(doc, allBucketsSlot);  // per-seg collectors
-      }
-    }
-  }
-
-}

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorNumeric.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorNumeric.java?rev=1690199&r1=1690198&r2=1690199&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorNumeric.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorNumeric.java Fri Jul 10 03:47:54 2015
@@ -25,17 +25,13 @@ import java.util.List;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.util.BitUtil;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.PriorityQueue;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.DocIterator;
-import org.apache.solr.search.DocSet;
 
 class FacetFieldProcessorNumeric extends FacetFieldProcessor {
   static int MAXIMUM_STARTING_TABLE_SIZE=1024;  // must be a power of two, non-final to support setting by tests
@@ -151,11 +147,11 @@ class FacetFieldProcessorNumeric extends
     response = calcFacets();
   }
 
-
   private void doRehash(LongCounts table) {
-    if (accs.length == 0) return;  // TODO: FUTURE: only need to resize acc we will sort on
+    if (collectAcc == null && missingAcc == null && allBucketsAcc == null) return;
 
     // Our "count" acc is backed by the hash table and will already be rehashed
+    // otherAccs don't need to be rehashed
 
     int newTableSize = table.numSlots();
     int numSlots = newTableSize;
@@ -164,7 +160,7 @@ class FacetFieldProcessorNumeric extends
     if (oldMissingSlot >= 0) {
       missingSlot = numSlots++;
     }
-    if (allBucketsSlot >= 0) {
+    if (oldAllBucketsSlot >= 0) {
       allBucketsSlot = numSlots++;
     }
 
@@ -192,8 +188,15 @@ class FacetFieldProcessorNumeric extends
       }
     };
 
-    for (SlotAcc acc : accs) {
-      acc.resize( resizer );
+    // NOTE: resizing isn't strictly necessary for missing/allBuckets... we could just set the new slot directly
+    if (collectAcc != null) {
+      collectAcc.resize(resizer);
+    }
+    if (missingAcc != null) {
+      missingAcc.resize(resizer);
+    }
+    if (allBucketsAcc != null) {
+      allBucketsAcc.resize(resizer);
     }
   }
 
@@ -292,12 +295,17 @@ class FacetFieldProcessorNumeric extends
       }
     };
 
+    // we set the countAcc & indexAcc first so generic ones won't be created for us.
+    createCollectAcc(fcontext.base.size(), numSlots);
 
-    // we set the countAcc first so it won't be created here
-    createAccs(fcontext.base.size(), numSlots);
-    setSortAcc(numSlots);
-    prepareForCollection();
+    if (freq.allBuckets) {
+      allBucketsAcc = new SpecialSlotAcc(fcontext, collectAcc, allBucketsSlot, otherAccs, 0);
+    }
 
+    if (freq.missing) {
+      // TODO: optimize case when missingSlot can be contiguous with other slots
+      missingAcc = new SpecialSlotAcc(fcontext, collectAcc, missingSlot, otherAccs, 1);
+    }
 
     NumericDocValues values = null;
     Bits docsWithField = null;
@@ -319,7 +327,7 @@ class FacetFieldProcessorNumeric extends
           adjustedMax = segBase + segMax;
         } while (doc >= adjustedMax);
         assert doc >= ctx.docBase;
-        setNextReader(ctx);
+        setNextReaderFirstPhase(ctx);
 
         values = DocValues.getNumeric(ctx.reader(), sf.getName());
         docsWithField = DocValues.getDocsWithField(ctx.reader(), sf.getName());
@@ -328,19 +336,16 @@ class FacetFieldProcessorNumeric extends
       int segDoc = doc - segBase;
       long val = values.get(segDoc);
       if (val == 0 && !docsWithField.get(segDoc)) {
-        // missing
-        if (missingSlot >= 0) {
-          numMissing++;
-          collect(segDoc, missingSlot);
+        if (missingAcc != null) {
+          missingAcc.collect(segDoc, -1);
         }
       } else {
         int slot = table.add(val);  // this can trigger a rehash rehash
 
-        collect(segDoc, slot);
+        // countAcc.incrementCount(slot, 1);
+        // our countAcc is virtual, so this is not needed
 
-        if (allBucketsSlot >= 0) {
-          collect(segDoc, allBucketsSlot);
-        }
+        collectFirstPhase(segDoc, slot);
       }
     }
 
@@ -414,29 +419,16 @@ class FacetFieldProcessorNumeric extends
       SimpleOrderedMap<Object> allBuckets = new SimpleOrderedMap<>();
       // countAcc.setValues(allBuckets, allBucketsSlot);
       allBuckets.add("count", table.numAdds);
-      for (SlotAcc acc : accs) {
-        acc.setValues(allBuckets, allBucketsSlot);
-      }
+      allBucketsAcc.setValues(allBuckets, -1);
       // allBuckets currently doesn't execute sub-facets (because it doesn't change the domain?)
       res.add("allBuckets", allBuckets);
     }
 
     if (freq.missing) {
-      SimpleOrderedMap<Object> missingBucket = new SimpleOrderedMap<>();
-      // countAcc.setValues(missingBucket, missingSlot);
-      missingBucket.add("count", numMissing);
-      for (SlotAcc acc : accs) {
-        acc.setValues(missingBucket, missingSlot);
-      }
+      // TODO: it would be more efficient to buid up a missing DocSet if we need it here anyway.
 
-      if (freq.getSubFacets().size() > 0) {
-        // TODO: we can do better than this!
-        DocSet missingDocSet = null;
-        if (missingDocSet == null) {
-          missingDocSet = getFieldMissing(fcontext.searcher, fcontext.base, freq.field);
-        }
-        processSubs(missingBucket, getFieldMissingQuery(fcontext.searcher, freq.field), missingDocSet);
-      }
+      SimpleOrderedMap<Object> missingBucket = new SimpleOrderedMap<>();
+      fillBucket(missingBucket, getFieldMissingQuery(fcontext.searcher, freq.field));
       res.add("missing", missingBucket);
     }
 
@@ -451,26 +443,16 @@ class FacetFieldProcessorNumeric extends
     ArrayList bucketList = new ArrayList(collectCount);
     res.add("buckets", bucketList);
 
+    boolean needFilter = deferredAggs != null || freq.getSubFacets().size() > 0;
 
     for (int slotNum : sortedSlots) {
       SimpleOrderedMap<Object> bucket = new SimpleOrderedMap<>();
       Comparable val = calc.bitsToValue(table.vals[slotNum]);
       bucket.add("val", val);
 
-      // add stats for this bucket
-      // TODO: this gets count from countAcc
-      // addStats(bucket, slotNum);
-      bucket.add("count", table.counts[slotNum]);
-
-      for (SlotAcc acc : accs) {
-        acc.setValues(bucket, slotNum);
-      }
+      Query filter = needFilter ? sf.getType().getFieldQuery(null, sf, calc.formatValue(val)) : null;
 
-      // handle sub-facets for this bucket
-      if (freq.getSubFacets().size() > 0) {
-        Query filter = sf.getType().getFieldQuery(null, sf, calc.formatValue(val));
-        processSubs(bucket, filter, fcontext.searcher.getDocSet(filter, fcontext.base) );
-      }
+      fillBucket(bucket, table.counts[slotNum], slotNum, null, filter);
 
       bucketList.add(bucket);
     }

Copied: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java (from r1690189, lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java?p2=lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java&p1=lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java&r1=1690189&r2=1690199&rev=1690199&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java Fri Jul 10 03:47:54 2015
@@ -329,9 +329,14 @@ public class FacetProcessor<FacetRequest
   public static Query getFieldMissingQuery(SolrIndexSearcher searcher, String fieldName) throws IOException {
     SchemaField sf = searcher.getSchema().getField(fieldName);
     Query hasVal = sf.getType().getRangeQuery(null, sf, null, null, false, false);
+    BooleanQuery noVal = new BooleanQuery();
+    noVal.add(hasVal, BooleanClause.Occur.MUST_NOT);
+    return noVal;
+    /*** Lucene 6 version... BooleanQuery.Builder is not yet part of Lucene 5
     BooleanQuery.Builder noVal = new BooleanQuery.Builder();
     noVal.add(hasVal, BooleanClause.Occur.MUST_NOT);
     return noVal.build();
+     ***/
   }
 
 }

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java?rev=1690199&r1=1690198&r2=1690199&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java Fri Jul 10 03:47:54 2015
@@ -265,7 +265,6 @@ class FacetRangeProcessor extends FacetP
 
 
     createAccs(fcontext.base.size(), slotCount);
-    prepareForCollection();
 
     for (int idx = 0; idx<rangeList.size(); idx++) {
       rangeStats(rangeList.get(idx), idx);

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java?rev=1690199&r1=1690198&r2=1690199&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java Fri Jul 10 03:47:54 2015
@@ -132,302 +132,6 @@ class FacetContext {
 }
 
 
-class FacetProcessor<FacetRequestT extends FacetRequest>  {
-  protected SimpleOrderedMap<Object> response;
-  protected FacetContext fcontext;
-  protected FacetRequestT freq;
-
-  LinkedHashMap<String,SlotAcc> accMap;
-  protected SlotAcc[] accs;
-  protected CountSlotAcc countAcc;
-
-  FacetProcessor(FacetContext fcontext, FacetRequestT freq) {
-    this.fcontext = fcontext;
-    this.freq = freq;
-  }
-
-  public void process() throws IOException {
-    handleDomainChanges();
-  }
-
-  protected void handleDomainChanges() throws IOException {
-    if (freq.domain == null) return;
-    handleFilterExclusions();
-    handleBlockJoin();
-  }
-
-  private void handleBlockJoin() throws IOException {
-    if (!(freq.domain.toChildren || freq.domain.toParent)) return;
-
-    // TODO: avoid query parsing per-bucket somehow...
-    String parentStr = freq.domain.parents;
-    Query parentQuery;
-    try {
-      QParser parser = QParser.getParser(parentStr, null, fcontext.req);
-      parentQuery = parser.getQuery();
-    } catch (SyntaxError err) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Error parsing block join parent specification: " + parentStr);
-    }
-
-    BitDocSet parents = fcontext.searcher.getDocSetBits(parentQuery);
-    DocSet input = fcontext.base;
-    DocSet result;
-
-    if (freq.domain.toChildren) {
-      DocSet filt = fcontext.searcher.getDocSetBits( new MatchAllDocsQuery() );
-      result = BlockJoin.toChildren(input, parents, filt, fcontext.qcontext);
-    } else {
-      result = BlockJoin.toParents(input, parents, fcontext.qcontext);
-    }
-
-    fcontext.base = result;
-  }
-
-  private void handleFilterExclusions() throws IOException {
-    List<String> excludeTags = freq.domain.excludeTags;
-
-    if (excludeTags == null || excludeTags.size() == 0) {
-      return;
-    }
-
-    // TODO: somehow remove responsebuilder dependency
-    ResponseBuilder rb = SolrRequestInfo.getRequestInfo().getResponseBuilder();
-    Map tagMap = (Map) rb.req.getContext().get("tags");
-    if (tagMap == null) {
-      // no filters were tagged
-      return;
-    }
-
-    IdentityHashMap<Query,Boolean> excludeSet = new IdentityHashMap<>();
-    for (String excludeTag : excludeTags) {
-      Object olst = tagMap.get(excludeTag);
-      // tagMap has entries of List<String,List<QParser>>, but subject to change in the future
-      if (!(olst instanceof Collection)) continue;
-      for (Object o : (Collection<?>)olst) {
-        if (!(o instanceof QParser)) continue;
-        QParser qp = (QParser)o;
-        try {
-          excludeSet.put(qp.getQuery(), Boolean.TRUE);
-        } catch (SyntaxError syntaxError) {
-          // This should not happen since we should only be retrieving a previously parsed query
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, syntaxError);
-        }
-      }
-    }
-    if (excludeSet.size() == 0) return;
-
-    List<Query> qlist = new ArrayList<>();
-
-    // add the base query
-    if (!excludeSet.containsKey(rb.getQuery())) {
-      qlist.add(rb.getQuery());
-    }
-
-    // add the filters
-    if (rb.getFilters() != null) {
-      for (Query q : rb.getFilters()) {
-        if (!excludeSet.containsKey(q)) {
-          qlist.add(q);
-        }
-      }
-    }
-
-    // now walk back up the context tree
-    // TODO: we lose parent exclusions...
-    for (FacetContext curr = fcontext; curr != null; curr = curr.parent) {
-      if (curr.filter != null) {
-        qlist.add( curr.filter );
-      }
-    }
-
-    // recompute the base domain
-    fcontext.base = fcontext.searcher.getDocSet(qlist);
-  }
-
-
-  public Object getResponse() {
-    return null;
-  }
-
-
-  protected void createAccs(int docCount, int slotCount) throws IOException {
-    accMap = new LinkedHashMap<String,SlotAcc>();
-
-    // allow a custom count acc to be used
-    if (countAcc == null) {
-      countAcc = new CountSlotArrAcc(fcontext, slotCount);
-      countAcc.key = "count";
-    }
-
-    for (Map.Entry<String,AggValueSource> entry : freq.getFacetStats().entrySet()) {
-      SlotAcc acc = entry.getValue().createSlotAcc(fcontext, docCount, slotCount);
-      acc.key = entry.getKey();
-      accMap.put(acc.key, acc);
-    }
-  }
-
-  /** Create the actual accs array from accMap before starting to collect stats. */
-  protected void prepareForCollection() {
-    accs = new SlotAcc[accMap.size()];
-    int i=0;
-    for (SlotAcc acc : accMap.values()) {
-      accs[i++] = acc;
-    }
-  }
-
-  protected void resetStats() {
-    countAcc.reset();
-    for (SlotAcc acc : accs) {
-      acc.reset();
-    }
-  }
-
-  protected void processStats(SimpleOrderedMap<Object> bucket, DocSet docs, int docCount) throws IOException {
-    if (docCount == 0 && !freq.processEmpty || freq.getFacetStats().size() == 0) {
-      bucket.add("count", docCount);
-      return;
-    }
-    createAccs(docCount, 1);
-    prepareForCollection();
-    int collected = collect(docs, 0);
-    countAcc.incrementCount(0, collected);
-    assert collected == docCount;
-    addStats(bucket, 0);
-  }
-
-
-  protected void processSubs(SimpleOrderedMap<Object> response, Query filter, DocSet domain) throws IOException {
-
-    // TODO: what if a zero bucket has a sub-facet with an exclusion that would yield results?
-    // should we check for domain-altering exclusions, or even ask the sub-facet for
-    // it's domain and then only skip it if it's 0?
-
-    if (domain == null || domain.size() == 0 && !freq.processEmpty) {
-      return;
-    }
-
-    for (Map.Entry<String,FacetRequest> sub : freq.getSubFacets().entrySet()) {
-      // make a new context for each sub-facet since they can change the domain
-      FacetContext subContext = fcontext.sub(filter, domain);
-      FacetProcessor subProcessor = sub.getValue().createFacetProcessor(subContext);
-      subProcessor.process();
-      response.add( sub.getKey(), subProcessor.getResponse() );
-    }
-  }
-
-  int collect(DocSet docs, int slot) throws IOException {
-    int count = 0;
-    SolrIndexSearcher searcher = fcontext.searcher;
-
-    final List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
-    final Iterator<LeafReaderContext> ctxIt = leaves.iterator();
-    LeafReaderContext ctx = null;
-    int segBase = 0;
-    int segMax;
-    int adjustedMax = 0;
-    for (DocIterator docsIt = docs.iterator(); docsIt.hasNext(); ) {
-      final int doc = docsIt.nextDoc();
-      if (doc >= adjustedMax) {
-        do {
-          ctx = ctxIt.next();
-          if (ctx == null) {
-            // should be impossible
-            throw new RuntimeException("INTERNAL FACET ERROR");
-          }
-          segBase = ctx.docBase;
-          segMax = ctx.reader().maxDoc();
-          adjustedMax = segBase + segMax;
-        } while (doc >= adjustedMax);
-        assert doc >= ctx.docBase;
-        setNextReader(ctx);
-      }
-      count++;
-      collect(doc - segBase, slot);  // per-seg collectors
-    }
-    return count;
-  }
-
-  void collect(int segDoc, int slot) throws IOException {
-    for (SlotAcc acc : accs) {
-      acc.collect(segDoc, slot);
-    }
-  }
-
-  void setNextReader(LeafReaderContext ctx) throws IOException {
-    // countAcc.setNextReader is a no-op
-    for (SlotAcc acc : accs) {
-      acc.setNextReader(ctx);
-    }
-  }
-
-  void addStats(SimpleOrderedMap<Object> target, int slotNum) throws IOException {
-    int count = countAcc.getCount(slotNum);
-    target.add("count", count);
-    if (count > 0 || freq.processEmpty) {
-      for (SlotAcc acc : accs) {
-        acc.setValues(target, slotNum);
-      }
-    }
-  }
-
-
-  public void fillBucket(SimpleOrderedMap<Object> bucket, Query q) throws IOException {
-    boolean needDocSet = freq.getFacetStats().size() > 0 || freq.getSubFacets().size() > 0;
-
-    // TODO: always collect counts or not???
-
-    DocSet result = null;
-    int count;
-
-    if (needDocSet) {
-      if (q == null) {
-        result = fcontext.base;
-        // result.incref(); // OFF-HEAP
-      } else {
-        result = fcontext.searcher.getDocSet(q, fcontext.base);
-      }
-      count = result.size();
-    } else {
-      if (q == null) {
-        count = fcontext.base.size();
-      } else {
-        count = fcontext.searcher.numDocs(q, fcontext.base);
-      }
-    }
-
-    try {
-      processStats(bucket, result, (int) count);
-      processSubs(bucket, q, result);
-    } finally {
-      if (result != null) {
-        // result.decref(); // OFF-HEAP
-        result = null;
-      }
-    }
-  }
-
-  public static DocSet getFieldMissing(SolrIndexSearcher searcher, DocSet docs, String fieldName) throws IOException {
-    SchemaField sf = searcher.getSchema().getField(fieldName);
-    DocSet hasVal = searcher.getDocSet(sf.getType().getRangeQuery(null, sf, null, null, false, false));
-    DocSet answer = docs.andNot(hasVal);
-    // hasVal.decref(); // OFF-HEAP
-    return answer;
-  }
-
-  public static Query getFieldMissingQuery(SolrIndexSearcher searcher, String fieldName) throws IOException {
-    SchemaField sf = searcher.getSchema().getField(fieldName);
-    Query hasVal = sf.getType().getRangeQuery(null, sf, null, null, false, false);
-    BooleanQuery noVal = new BooleanQuery();
-    noVal.add(hasVal, BooleanClause.Occur.MUST_NOT);
-    return noVal;
-  }
-
-}
-
-
-
-
-
 abstract class FacetParser<FacetRequestT extends FacetRequest> {
   protected FacetRequestT facet;
   protected FacetParser parent;

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java?rev=1690199&r1=1690198&r2=1690199&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java Fri Jul 10 03:47:54 2015
@@ -29,6 +29,8 @@ import org.apache.lucene.util.FixedBitSe
 import org.apache.lucene.util.LongValues;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.DocIterator;
+import org.apache.solr.search.DocSet;
 import org.apache.solr.search.SolrIndexSearcher;
 
 import java.io.Closeable;
@@ -36,6 +38,7 @@ import java.io.IOException;
 import java.lang.reflect.Array;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Iterator;
 import java.util.List;
 
 
@@ -52,6 +55,39 @@ public abstract class SlotAcc implements
 
   public abstract void collect(int doc, int slot) throws IOException;
 
+  public int collect(DocSet docs, int slot) throws IOException {
+    int count = 0;
+    SolrIndexSearcher searcher = fcontext.searcher;
+
+    final List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
+    final Iterator<LeafReaderContext> ctxIt = leaves.iterator();
+    LeafReaderContext ctx = null;
+    int segBase = 0;
+    int segMax;
+    int adjustedMax = 0;
+    for (DocIterator docsIt = docs.iterator(); docsIt.hasNext(); ) {
+      final int doc = docsIt.nextDoc();
+      if (doc >= adjustedMax) {
+        do {
+          ctx = ctxIt.next();
+          if (ctx == null) {
+            // should be impossible
+            throw new RuntimeException("INTERNAL FACET ERROR");
+          }
+          segBase = ctx.docBase;
+          segMax = ctx.reader().maxDoc();
+          adjustedMax = segBase + segMax;
+        } while (doc >= adjustedMax);
+        assert doc >= ctx.docBase;
+        setNextReader(ctx);
+      }
+      count++;
+      collect(doc - segBase, slot);  // per-seg collectors
+    }
+    return count;
+  }
+
+
   public abstract int compare(int slotA, int slotB);
 
   public abstract Object getValue(int slotNum) throws IOException;

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/UnInvertedField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/UnInvertedField.java?rev=1690199&r1=1690198&r2=1690199&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/UnInvertedField.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/UnInvertedField.java Fri Jul 10 03:47:54 2015
@@ -240,7 +240,7 @@ public class UnInvertedField extends Doc
       return te;
     }
 
-    public void getTerms(int doc, Callback target) throws IOException {
+    public void getBigTerms(int doc, Callback target) throws IOException {
       if (bigTermSets != null) {
         for (int i=0; i<bigTermSets.length; i++) {
           if (bigTermSets[i].exists(doc)) {
@@ -248,12 +248,9 @@ public class UnInvertedField extends Doc
           }
         }
       }
-
-      getNonBigTerms(doc, target);
     }
 
-
-    public void getNonBigTerms(int doc, Callback target) {
+    public void getSmallTerms(int doc, Callback target) {
       if (termInstances > 0) {
         int code = index[doc];
 
@@ -309,6 +306,7 @@ public class UnInvertedField extends Doc
     int baseSize = docs.size();
     int maxDoc = searcher.maxDoc();
 
+    // what about allBuckets?
     if (baseSize < processor.effectiveMincount) {
       return;
     }
@@ -382,6 +380,7 @@ public class UnInvertedField extends Doc
       }
     }
 
+    /*** TODO - future optimization to handle allBuckets
     if (processor.allBucketsSlot >= 0) {
       int all = 0;  // overflow potential
       for (int i=0; i<numTermsInField; i++) {
@@ -389,33 +388,14 @@ public class UnInvertedField extends Doc
       }
       counts.incrementCount(processor.allBucketsSlot, all);
     }
+     ***/
   }
 
 
 
   public void collectDocs(FacetFieldProcessorUIF processor) throws IOException {
-    if (processor.accs.length == 0 && processor.startTermIndex == 0 && processor.endTermIndex >= numTermsInField)
-    {
+    if (processor.collectAcc==null && processor.missingAcc == null && processor.allBucketsAcc == null && processor.startTermIndex == 0 && processor.endTermIndex >= numTermsInField) {
       getCounts(processor, processor.countAcc);
-
-      /*** debugging
-      int sz = processor.countAcc.getCountArray().length;
-      CountSlotAcc acc = processor.countAcc;
-      CountSlotAcc acc2 = new CountSlotAcc(processor.fcontext, sz);
-      processor.countAcc = acc2;
-      collectDocsGeneric(processor); // hopefully we can call this again?
-
-      for (int i=0; i<sz; i++) {
-        if (acc.getCount(i) != acc2.getCount(i)) {
-          System.out.println("ERROR! ERROR! i=" + i + " counts=" + acc.getCount(i) + " " + acc2.getCount(i));
-          CountSlotAcc acc3 = new CountSlotAcc(processor.fcontext, sz);  // put breakpoint here and re-execute
-          processor.countAcc = acc3;
-          int[] arr3 = processor.countAcc.getCountArray();
-          getCountsInArray(processor, arr3);
-        }
-      }
-       ***/
-
       return;
     }
 
@@ -433,18 +413,15 @@ public class UnInvertedField extends Doc
     DocSet docs = processor.fcontext.base;
 
     int uniqueTerms = 0;
+    final CountSlotAcc countAcc = processor.countAcc;
 
     for (TopTerm tt : bigTerms.values()) {
       if (tt.termNum >= startTermIndex && tt.termNum < endTermIndex) {
         // handle the biggest terms
         try ( DocSet intersection = searcher.getDocSet(tt.termQuery, docs); )
         {
-          int collected = processor.collect(intersection, tt.termNum - startTermIndex);
-          processor.countAcc.incrementCount(tt.termNum - startTermIndex, collected);
-          if (processor.allBucketsSlot >= 0) {
-            processor.collect(intersection, processor.allBucketsSlot);
-            processor.countAcc.incrementCount(processor.allBucketsSlot, collected);
-          }
+          int collected = processor.collectFirstPhase(intersection, tt.termNum - startTermIndex);
+          countAcc.incrementCount(tt.termNum - startTermIndex, collected);
           if (collected > 0) {
             uniqueTerms++;
           }
@@ -452,6 +429,7 @@ public class UnInvertedField extends Doc
       }
     }
 
+
     if (termInstances > 0) {
 
       final List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
@@ -480,7 +458,7 @@ public class UnInvertedField extends Doc
             adjustedMax = segBase + segMax;
           } while (doc >= adjustedMax);
           assert doc >= ctx.docBase;
-          processor.setNextReader(ctx);
+          processor.setNextReaderFirstPhase(ctx);
         }
         int segDoc = doc - segBase;
 
@@ -504,12 +482,8 @@ public class UnInvertedField extends Doc
             int arrIdx = tnum - startTermIndex;
             if (arrIdx < 0) continue;
             if (arrIdx >= nTerms) break;
-            processor.countAcc.incrementCount(arrIdx, 1);
-            processor.collect(segDoc, arrIdx);
-            if (processor.allBucketsSlot >= 0) {
-              processor.countAcc.incrementCount(processor.allBucketsSlot, 1);
-              processor.collect(segDoc, processor.allBucketsSlot);
-            }
+            countAcc.incrementCount(arrIdx, 1);
+            processor.collectFirstPhase(segDoc, arrIdx);
           }
         } else {
           int tnum = 0;
@@ -522,13 +496,8 @@ public class UnInvertedField extends Doc
               int arrIdx = tnum - startTermIndex;
               if (arrIdx < 0) continue;
               if (arrIdx >= nTerms) break;
-              processor.countAcc.incrementCount(arrIdx, 1);
-              processor.collect(segDoc, arrIdx);
-              if (processor.allBucketsSlot >= 0) {
-                processor.countAcc.incrementCount(processor.allBucketsSlot, 1);
-                processor.collect(segDoc, processor.allBucketsSlot);
-              }
-
+              countAcc.incrementCount(arrIdx, 1);
+              processor.collectFirstPhase(segDoc, arrIdx);
               delta = 0;
             }
             code >>>= 8;

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/UniqueSlotAcc.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/UniqueSlotAcc.java?rev=1690199&r1=1690198&r2=1690199&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/UniqueSlotAcc.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/facet/UniqueSlotAcc.java Fri Jul 10 03:47:54 2015
@@ -311,7 +311,8 @@ class UniqueMultivaluedSlotAcc extends U
       bits = new FixedBitSet(nTerms);
       arr[slotNum] = bits;
     }
-    docToTerm.getTerms(doc + currentDocBase, this);  // this will call back to our Callback.call(int termNum)
+    docToTerm.getBigTerms(doc + currentDocBase, this);  // this will call back to our Callback.call(int termNum)
+    docToTerm.getSmallTerms(doc + currentDocBase, this);
   }
 
   @Override