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/05/17 18:07:34 UTC

svn commit: r1679876 - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/search/ core/src/java/org/apache/solr/search/facet/ core/src/test/org/apache/solr/search/facet/ solrj/src/java/org/apache/solr/common/util/

Author: yonik
Date: Sun May 17 16:07:33 2015
New Revision: 1679876

URL: http://svn.apache.org/r1679876
Log:
SOLR-7553: hll function for hyperloglog

Added:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java   (with props)
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/UniqueSlotAcc.java   (with props)
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/util/Hash.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1679876&r1=1679875&r2=1679876&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Sun May 17 16:07:33 2015
@@ -190,6 +190,12 @@ New Features
 
 * SOLR-7377: Solr Streaming Expressions (Dennis Gove, Joel Bernstein, Steven Bower)
 
+* SOLR-7553: Facet Analytics Module: new "hll" function that uses HyperLogLog to calculate
+  distributed cardinality.  The original "unique" function is still available.
+  Example:  json.facet={ numProducts : "hll(product_id)" }
+  (yonik)
+
+
 Bug Fixes
 ----------------------
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java?rev=1679876&r1=1679875&r2=1679876&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java Sun May 17 16:07:33 2015
@@ -43,6 +43,7 @@ import org.apache.solr.schema.*;
 import org.apache.solr.search.facet.AggValueSource;
 import org.apache.solr.search.facet.AvgAgg;
 import org.apache.solr.search.facet.CountAgg;
+import org.apache.solr.search.facet.HLLAgg;
 import org.apache.solr.search.facet.MaxAgg;
 import org.apache.solr.search.facet.MinAgg;
 import org.apache.solr.search.facet.PercentileAgg;
@@ -818,6 +819,13 @@ public abstract class ValueSourceParser
       }
     });
 
+    addParser("agg_hll", new ValueSourceParser() {
+      @Override
+      public ValueSource parse(FunctionQParser fp) throws SyntaxError {
+        return new HLLAgg(fp.parseArg());
+      }
+    });
+
     addParser("agg_sum", new ValueSourceParser() {
       @Override
       public ValueSource parse(FunctionQParser fp) throws SyntaxError {

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java?rev=1679876&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java Sun May 17 16:07:33 2015
@@ -0,0 +1,191 @@
+package org.apache.solr.search.facet;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import net.agkn.hll.HLL;
+import net.agkn.hll.HLLType;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.util.Bits;
+import org.apache.solr.common.util.Hash;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.schema.SchemaField;
+
+public class HLLAgg extends StrAggValueSource {
+  protected HLLFactory factory;
+
+  public HLLAgg(String field) {
+    super("hll", field);
+    factory = new HLLFactory();
+  }
+
+  // factory for the hyper-log-log algorithm.
+  // TODO: make stats component HllOptions inherit from this?
+  public static class HLLFactory {
+    int log2m = 13;
+    int regwidth = 6;
+    public HLL getHLL() {
+      return new HLL(log2m, regwidth, -1 /* auto explict threshold */,
+          false /* no sparse representation */, HLLType.EMPTY);
+    }
+  }
+
+  @Override
+  public SlotAcc createSlotAcc(FacetContext fcontext, int numDocs, int numSlots) throws IOException {
+    SchemaField sf = fcontext.qcontext.searcher().getSchema().getField(getArg());
+    if (sf.multiValued() || sf.getType().multiValuedFieldCache()) {
+      if (sf.hasDocValues()) {
+        return new UniqueMultiDvSlotAcc(fcontext, getArg(), numSlots, fcontext.isShard() ? factory : null);
+      } else {
+        return new UniqueMultivaluedSlotAcc(fcontext, getArg(), numSlots, fcontext.isShard() ? factory : null);
+      }
+    } else {
+      if (sf.getType().getNumericType() != null) {
+        // always use hll here since we don't know how many values there are?
+        return new NumericAcc(fcontext, getArg(), numSlots);
+      } else {
+        return new UniqueSinglevaluedSlotAcc(fcontext, getArg(), numSlots, fcontext.isShard() ? factory : null);
+      }
+    }
+  }
+
+  @Override
+  public FacetMerger createFacetMerger(Object prototype) {
+    return new Merger();
+  }
+
+  private static class Merger extends FacetSortableMerger {
+    HLL aggregate = null;
+    long answer = -1;
+
+    @Override
+    public void merge(Object facetResult) {
+      SimpleOrderedMap map = (SimpleOrderedMap)facetResult;
+      byte[] serialized = ((byte[])map.get("hll"));
+      HLL subHLL = HLL.fromBytes(serialized);
+      if (aggregate == null) {
+        aggregate = subHLL;
+      } else {
+        aggregate.union(subHLL);
+      }
+    }
+
+    private long getLong() {
+      if (answer < 0) {
+        answer = aggregate.cardinality();
+      }
+      return answer;
+    }
+
+    @Override
+    public Object getMergedResult() {
+      return getLong();
+    }
+
+    @Override
+    public int compareTo(FacetSortableMerger other, FacetField.SortDirection direction) {
+      return Long.compare( getLong(), ((Merger)other).getLong() );
+    }
+  }
+
+
+  // TODO: hybrid model for non-distrib numbers?
+  // todo - better efficiency for sorting?
+
+  class NumericAcc extends SlotAcc {
+    SchemaField sf;
+    HLL[] sets;
+    NumericDocValues values;
+    Bits exists;
+
+    public NumericAcc(FacetContext fcontext, String field, int numSlots) throws IOException {
+      super(fcontext);
+      sf = fcontext.searcher.getSchema().getField(field);
+      sets = new HLL[numSlots];
+    }
+
+    @Override
+    public void reset() {
+      sets = new HLL[sets.length];
+    }
+
+    @Override
+    public void resize(Resizer resizer) {
+      resizer.resize(sets, null);
+    }
+
+    @Override
+    public void setNextReader(LeafReaderContext readerContext) throws IOException {
+      values = DocValues.getNumeric(readerContext.reader(),  sf.getName());
+      exists = DocValues.getDocsWithField(readerContext.reader(), sf.getName());
+    }
+
+    @Override
+    public void collect(int doc, int slot) throws IOException {
+      long val = values.get(doc);
+      if (val == 0 && !exists.get(doc)) {
+        return;
+      }
+
+      long hash = Hash.fmix64(val);
+
+      HLL hll = sets[slot];
+      if (hll == null) {
+        hll = sets[slot] = factory.getHLL();
+      }
+      hll.addRaw(hash);
+    }
+
+    @Override
+    public Object getValue(int slot) throws IOException {
+      if (fcontext.isShard()) {
+        return getShardValue(slot);
+      }
+      return getCardinality(slot);
+    }
+
+    private int getCardinality(int slot) {
+      HLL set = sets[slot];
+      return set==null ? 0 : (int)set.cardinality();
+    }
+
+    public Object getShardValue(int slot) throws IOException {
+      HLL hll = sets[slot];
+      if (hll == null) return null;
+      SimpleOrderedMap map = new SimpleOrderedMap();
+      map.add("hll", hll.toBytes());
+      // optionally use explicit values
+      return map;
+    }
+
+    @Override
+    public int compare(int slotA, int slotB) {
+      return getCardinality(slotA) - getCardinality(slotB);
+    }
+
+  }
+
+
+}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java?rev=1679876&r1=1679875&r2=1679876&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java Sun May 17 16:07:33 2015
@@ -436,266 +436,4 @@ class SortSlotAcc extends SlotAcc {
     // sort slot only works with direct-mapped accumulators
     throw new UnsupportedOperationException();
   }
-}
-
-
-abstract class UniqueSlotAcc extends SlotAcc {
-  SchemaField field;
-  FixedBitSet[] arr;
-  int currentDocBase;
-  int[] counts;  // populated with the cardinality once
-  int nTerms;
-
-  public UniqueSlotAcc(FacetContext fcontext, String field, int numSlots) throws IOException {
-    super(fcontext);
-    arr = new FixedBitSet[numSlots];
-    this.field = fcontext.searcher.getSchema().getField(field);
-  }
-
-  @Override
-  public void reset() {
-    counts = null;
-    for (FixedBitSet bits : arr) {
-      if (bits == null) continue;
-      bits.clear(0, bits.length());
-    }
-  }
-
-  @Override
-  public void setNextReader(LeafReaderContext readerContext) throws IOException {
-    currentDocBase = readerContext.docBase;
-  }
-
-  @Override
-  public Object getValue(int slot) throws IOException {
-    if (fcontext.isShard()) {
-      return getShardValue(slot);
-    }
-    if (counts != null) {  // will only be pre-populated if this was used for sorting.
-      return counts[slot];
-    }
-
-    FixedBitSet bs = arr[slot];
-    return bs==null ? 0 : bs.cardinality();
-  }
-
-  public Object getShardValue(int slot) throws IOException {
-    FixedBitSet ords = arr[slot];
-    int unique;
-    if (counts != null) {
-      unique = counts[slot];
-    } else {
-      unique = ords==null ? 0 : ords.cardinality();
-    }
-
-    SimpleOrderedMap map = new SimpleOrderedMap();
-    map.add("unique", unique);
-    map.add("nTerms", nTerms);
-
-    int maxExplicit=100;
-    // TODO: make configurable
-    // TODO: share values across buckets
-    if (unique > 0) {
-
-      List lst = new ArrayList( Math.min(unique, maxExplicit) );
-
-      long maxOrd = ords.length();
-      if (ords != null && ords.length() > 0) {
-        for (int ord=0; lst.size() < maxExplicit;) {
-          ord = ords.nextSetBit(ord);
-          if (ord == DocIdSetIterator.NO_MORE_DOCS) break;
-          BytesRef val = lookupOrd(ord);
-          Object o = field.getType().toObject(field, val);
-          lst.add(o);
-          if (++ord >= maxOrd) break;
-        }
-      }
-
-      map.add("vals", lst);
-    }
-
-    return map;
-  }
-
-  protected abstract BytesRef lookupOrd(int ord) throws IOException;
-
-  // we only calculate all the counts when sorting by count
-  public void calcCounts() {
-    counts = new int[arr.length];
-    for (int i=0; i<arr.length; i++) {
-      FixedBitSet bs = arr[i];
-      counts[i] = bs == null ? 0 : bs.cardinality();
-    }
-  }
-
-  @Override
-  public int compare(int slotA, int slotB) {
-    if (counts == null) {  // TODO: a more efficient way to do this?  prepareSort?
-      calcCounts();
-    }
-    return counts[slotA] - counts[slotB];
-  }
-
-  @Override
-  public void resize(Resizer resizer) {
-    arr = resizer.resize(arr, null);
-  }
-}
-
-
-class UniqueSinglevaluedSlotAcc extends UniqueSlotAcc {
-  final SortedDocValues topLevel;
-  final SortedDocValues[] subDvs;
-  final MultiDocValues.OrdinalMap ordMap;
-  LongValues toGlobal;
-  SortedDocValues subDv;
-
-  public UniqueSinglevaluedSlotAcc(FacetContext fcontext, String field, int numSlots) throws IOException {
-    super(fcontext, field, numSlots);
-    SolrIndexSearcher searcher = fcontext.qcontext.searcher();
-    topLevel = FieldUtil.getSortedDocValues(fcontext.qcontext, searcher.getSchema().getField(field), null);
-    nTerms = topLevel.getValueCount();
-    if (topLevel instanceof MultiDocValues.MultiSortedDocValues) {
-      ordMap = ((MultiDocValues.MultiSortedDocValues)topLevel).mapping;
-      subDvs = ((MultiDocValues.MultiSortedDocValues)topLevel).values;
-    } else {
-      ordMap = null;
-      subDvs = null;
-    }
-  }
-
-  @Override
-  protected BytesRef lookupOrd(int ord) {
-    return topLevel.lookupOrd(ord);
-  }
-
-  @Override
-  public void setNextReader(LeafReaderContext readerContext) throws IOException {
-    super.setNextReader(readerContext);
-    if (subDvs != null) {
-      subDv = subDvs[readerContext.ord];
-      toGlobal = ordMap.getGlobalOrds(readerContext.ord);
-    } else {
-      assert readerContext.ord==0 || topLevel.getValueCount() == 0;
-      subDv = topLevel;
-    }
-  }
-
-  @Override
-  public void collect(int doc, int slotNum) {
-    int segOrd = subDv.getOrd(doc);
-    if (segOrd < 0) return;  // -1 means missing
-    int ord = toGlobal==null ? segOrd : (int)toGlobal.get(segOrd);
-
-    FixedBitSet bits = arr[slotNum];
-    if (bits == null) {
-      bits = new FixedBitSet(nTerms);
-      arr[slotNum] = bits;
-    }
-    bits.set(ord);
-  }
-}
-
-
-class UniqueMultiDvSlotAcc extends UniqueSlotAcc {
-  final SortedSetDocValues topLevel;
-  final SortedSetDocValues[] subDvs;
-  final MultiDocValues.OrdinalMap ordMap;
-  LongValues toGlobal;
-  SortedSetDocValues subDv;
-
-  public UniqueMultiDvSlotAcc(FacetContext fcontext, String field, int numSlots) throws IOException {
-    super(fcontext, field, numSlots);
-    SolrIndexSearcher searcher = fcontext.qcontext.searcher();
-    topLevel = FieldUtil.getSortedSetDocValues(fcontext.qcontext, searcher.getSchema().getField(field), null);
-    nTerms = (int) topLevel.getValueCount();
-    if (topLevel instanceof MultiDocValues.MultiSortedSetDocValues) {
-      ordMap = ((MultiDocValues.MultiSortedSetDocValues) topLevel).mapping;
-      subDvs = ((MultiDocValues.MultiSortedSetDocValues) topLevel).values;
-    } else {
-      ordMap = null;
-      subDvs = null;
-    }
-  }
-
-  @Override
-  protected BytesRef lookupOrd(int ord) {
-    return topLevel.lookupOrd(ord);
-  }
-
-  @Override
-  public void setNextReader(LeafReaderContext readerContext) throws IOException {
-    super.setNextReader(readerContext);
-    if (subDvs != null) {
-      subDv = subDvs[readerContext.ord];
-      toGlobal = ordMap.getGlobalOrds(readerContext.ord);
-    } else {
-      assert readerContext.ord==0 || topLevel.getValueCount() == 0;
-      subDv = topLevel;
-    }
-  }
-
-  @Override
-  public void collect(int doc, int slotNum) {
-    subDv.setDocument(doc);
-    int segOrd = (int) subDv.nextOrd();
-    if (segOrd < 0) return;
-
-    FixedBitSet bits = arr[slotNum];
-    if (bits == null) {
-      bits = new FixedBitSet(nTerms);
-      arr[slotNum] = bits;
-    }
-
-    do {
-      int ord = toGlobal == null ? segOrd : (int) toGlobal.get(segOrd);
-      bits.set(ord);
-      segOrd = (int) subDv.nextOrd();
-    } while (segOrd >= 0);
-  }
-}
-
-
-
-class UniqueMultivaluedSlotAcc extends UniqueSlotAcc implements UnInvertedField.Callback {
-  private UnInvertedField uif;
-  private UnInvertedField.DocToTerm docToTerm;
-
-  public UniqueMultivaluedSlotAcc(FacetContext fcontext, String field, int numSlots) throws IOException {
-    super(fcontext, field, numSlots);
-    SolrIndexSearcher searcher = fcontext.qcontext.searcher();
-    uif = UnInvertedField.getUnInvertedField(field, searcher);
-    docToTerm = uif.new DocToTerm();
-    fcontext.qcontext.addCloseHook(this);  // TODO: find way to close accumulators instead of using close hook?
-    nTerms = uif.numTerms();
-  }
-
-  @Override
-  protected BytesRef lookupOrd(int ord) throws IOException {
-    return docToTerm.lookupOrd(ord);
-  }
-
-  private FixedBitSet bits;  // bits for the current slot, only set for the callback
-  @Override
-  public void call(int termNum) {
-    bits.set(termNum);
-  }
-
-  @Override
-  public void collect(int doc, int slotNum) throws IOException {
-    bits = arr[slotNum];
-    if (bits == null) {
-      bits = new FixedBitSet(nTerms);
-      arr[slotNum] = bits;
-    }
-    docToTerm.getTerms(doc + currentDocBase, this);  // this will call back to our Callback.call(int termNum)
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (docToTerm != null) {
-      docToTerm.close();
-      docToTerm = null;
-    }
-  }
 }
\ No newline at end of file

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java?rev=1679876&r1=1679875&r2=1679876&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java Sun May 17 16:07:33 2015
@@ -26,10 +26,7 @@ import java.util.Set;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.FixedBitSet;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.schema.SchemaField;
 
@@ -48,15 +45,15 @@ public class UniqueAgg extends StrAggVal
     SchemaField sf = fcontext.qcontext.searcher().getSchema().getField(getArg());
     if (sf.multiValued() || sf.getType().multiValuedFieldCache()) {
       if (sf.hasDocValues()) {
-        return new UniqueMultiDvSlotAcc(fcontext, getArg(), numSlots);
+        return new UniqueMultiDvSlotAcc(fcontext, getArg(), numSlots, null);
       } else {
-        return new UniqueMultivaluedSlotAcc(fcontext, getArg(), numSlots);
+        return new UniqueMultivaluedSlotAcc(fcontext, getArg(), numSlots, null);
       }
     } else {
       if (sf.getType().getNumericType() != null) {
         return new NumericAcc(fcontext, getArg(), numSlots);
       } else {
-        return new UniqueSinglevaluedSlotAcc(fcontext, getArg(), numSlots);
+        return new UniqueSinglevaluedSlotAcc(fcontext, getArg(), numSlots, null);
       }
     }
   }

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/UniqueSlotAcc.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/UniqueSlotAcc.java?rev=1679876&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/UniqueSlotAcc.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/facet/UniqueSlotAcc.java Sun May 17 16:07:33 2015
@@ -0,0 +1,324 @@
+package org.apache.solr.search.facet;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import net.agkn.hll.HLL;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.MultiDocValues;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.LongValues;
+import org.apache.solr.common.util.Hash;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.SolrIndexSearcher;
+
+abstract class UniqueSlotAcc extends SlotAcc {
+  HLLAgg.HLLFactory factory;
+  SchemaField field;
+  FixedBitSet[] arr;
+  int currentDocBase;
+  int[] counts;  // populated with the cardinality once
+  int nTerms;
+
+  public UniqueSlotAcc(FacetContext fcontext, String field, int numSlots, HLLAgg.HLLFactory factory) throws IOException {
+    super(fcontext);
+    this.factory = factory;
+    arr = new FixedBitSet[numSlots];
+    this.field = fcontext.searcher.getSchema().getField(field);
+  }
+
+  @Override
+  public void reset() {
+    counts = null;
+    for (FixedBitSet bits : arr) {
+      if (bits == null) continue;
+      bits.clear(0, bits.length());
+    }
+  }
+
+  @Override
+  public void setNextReader(LeafReaderContext readerContext) throws IOException {
+    currentDocBase = readerContext.docBase;
+  }
+
+  @Override
+  public Object getValue(int slot) throws IOException {
+    if (fcontext.isShard()) {
+      return getShardValue(slot);
+    }
+    if (counts != null) {  // will only be pre-populated if this was used for sorting.
+      return counts[slot];
+    }
+
+    FixedBitSet bs = arr[slot];
+    return bs==null ? 0 : bs.cardinality();
+  }
+
+  private Object getShardHLL(int slot) throws IOException {
+    FixedBitSet ords = arr[slot];
+    if (ords == null) return null;  // TODO: when we get to refinements, may be useful to return something???
+
+    HLL hll = factory.getHLL();
+    long maxOrd = ords.length();
+    Hash.LongPair hashResult = new Hash.LongPair();
+    for(int ord=-1; ++ord < maxOrd;) {
+      ord = ords.nextSetBit(ord);
+      if (ord == DocIdSetIterator.NO_MORE_DOCS) break;
+      BytesRef val = lookupOrd(ord);
+      // way to avoid recomputing hash across slots?  Prob not worth space
+      Hash.murmurhash3_x64_128(val.bytes, val.offset, val.length, 0, hashResult);
+      // idea: if the set is small enough, just send the hashes?  We can add at the top
+      // level or even just do a hash table at the top level.
+      hll.addRaw(hashResult.val1);
+    }
+
+    SimpleOrderedMap map = new SimpleOrderedMap();
+    map.add("hll", hll.toBytes());
+    return map;
+  }
+
+  private Object getShardValue(int slot) throws IOException {
+    if (factory != null) return getShardHLL(slot);
+    FixedBitSet ords = arr[slot];
+    int unique;
+    if (counts != null) {
+      unique = counts[slot];
+    } else {
+      unique = ords==null ? 0 : ords.cardinality();
+    }
+
+    SimpleOrderedMap map = new SimpleOrderedMap();
+    map.add("unique", unique);
+    map.add("nTerms", nTerms);
+
+    int maxExplicit=100;
+    // TODO: make configurable
+    // TODO: share values across buckets
+    if (unique > 0) {
+
+      List lst = new ArrayList( Math.min(unique, maxExplicit) );
+
+      long maxOrd = ords.length();
+      if (ords != null && ords.length() > 0) {
+        for (int ord=0; lst.size() < maxExplicit;) {
+          ord = ords.nextSetBit(ord);
+          if (ord == DocIdSetIterator.NO_MORE_DOCS) break;
+          BytesRef val = lookupOrd(ord);
+          Object o = field.getType().toObject(field, val);
+          lst.add(o);
+          if (++ord >= maxOrd) break;
+        }
+      }
+
+      map.add("vals", lst);
+    }
+
+    return map;
+  }
+
+  protected abstract BytesRef lookupOrd(int ord) throws IOException;
+
+  // we only calculate all the counts when sorting by count
+  public void calcCounts() {
+    counts = new int[arr.length];
+    for (int i=0; i<arr.length; i++) {
+      FixedBitSet bs = arr[i];
+      counts[i] = bs == null ? 0 : bs.cardinality();
+    }
+  }
+
+  @Override
+  public int compare(int slotA, int slotB) {
+    if (counts == null) {  // TODO: a more efficient way to do this?  prepareSort?
+      calcCounts();
+    }
+    return counts[slotA] - counts[slotB];
+  }
+
+  @Override
+  public void resize(Resizer resizer) {
+    arr = resizer.resize(arr, null);
+  }
+}
+
+
+class UniqueSinglevaluedSlotAcc extends UniqueSlotAcc {
+  final SortedDocValues topLevel;
+  final SortedDocValues[] subDvs;
+  final MultiDocValues.OrdinalMap ordMap;
+  LongValues toGlobal;
+  SortedDocValues subDv;
+
+  public UniqueSinglevaluedSlotAcc(FacetContext fcontext, String field, int numSlots, HLLAgg.HLLFactory factory) throws IOException {
+    super(fcontext, field, numSlots, factory);
+    SolrIndexSearcher searcher = fcontext.qcontext.searcher();
+    topLevel = FieldUtil.getSortedDocValues(fcontext.qcontext, searcher.getSchema().getField(field), null);
+    nTerms = topLevel.getValueCount();
+    if (topLevel instanceof MultiDocValues.MultiSortedDocValues) {
+      ordMap = ((MultiDocValues.MultiSortedDocValues)topLevel).mapping;
+      subDvs = ((MultiDocValues.MultiSortedDocValues)topLevel).values;
+    } else {
+      ordMap = null;
+      subDvs = null;
+    }
+  }
+
+  @Override
+  protected BytesRef lookupOrd(int ord) {
+    return topLevel.lookupOrd(ord);
+  }
+
+  @Override
+  public void setNextReader(LeafReaderContext readerContext) throws IOException {
+    super.setNextReader(readerContext);
+    if (subDvs != null) {
+      subDv = subDvs[readerContext.ord];
+      toGlobal = ordMap.getGlobalOrds(readerContext.ord);
+    } else {
+      assert readerContext.ord==0 || topLevel.getValueCount() == 0;
+      subDv = topLevel;
+    }
+  }
+
+  @Override
+  public void collect(int doc, int slotNum) {
+    int segOrd = subDv.getOrd(doc);
+    if (segOrd < 0) return;  // -1 means missing
+    int ord = toGlobal==null ? segOrd : (int)toGlobal.get(segOrd);
+
+    FixedBitSet bits = arr[slotNum];
+    if (bits == null) {
+      bits = new FixedBitSet(nTerms);
+      arr[slotNum] = bits;
+    }
+    bits.set(ord);
+  }
+}
+
+
+class UniqueMultiDvSlotAcc extends UniqueSlotAcc {
+  final SortedSetDocValues topLevel;
+  final SortedSetDocValues[] subDvs;
+  final MultiDocValues.OrdinalMap ordMap;
+  LongValues toGlobal;
+  SortedSetDocValues subDv;
+
+  public UniqueMultiDvSlotAcc(FacetContext fcontext, String field, int numSlots, HLLAgg.HLLFactory factory) throws IOException {
+    super(fcontext, field, numSlots, factory);
+    SolrIndexSearcher searcher = fcontext.qcontext.searcher();
+    topLevel = FieldUtil.getSortedSetDocValues(fcontext.qcontext, searcher.getSchema().getField(field), null);
+    nTerms = (int) topLevel.getValueCount();
+    if (topLevel instanceof MultiDocValues.MultiSortedSetDocValues) {
+      ordMap = ((MultiDocValues.MultiSortedSetDocValues) topLevel).mapping;
+      subDvs = ((MultiDocValues.MultiSortedSetDocValues) topLevel).values;
+    } else {
+      ordMap = null;
+      subDvs = null;
+    }
+  }
+
+  @Override
+  protected BytesRef lookupOrd(int ord) {
+    return topLevel.lookupOrd(ord);
+  }
+
+  @Override
+  public void setNextReader(LeafReaderContext readerContext) throws IOException {
+    super.setNextReader(readerContext);
+    if (subDvs != null) {
+      subDv = subDvs[readerContext.ord];
+      toGlobal = ordMap.getGlobalOrds(readerContext.ord);
+    } else {
+      assert readerContext.ord==0 || topLevel.getValueCount() == 0;
+      subDv = topLevel;
+    }
+  }
+
+  @Override
+  public void collect(int doc, int slotNum) {
+    subDv.setDocument(doc);
+    int segOrd = (int) subDv.nextOrd();
+    if (segOrd < 0) return;
+
+    FixedBitSet bits = arr[slotNum];
+    if (bits == null) {
+      bits = new FixedBitSet(nTerms);
+      arr[slotNum] = bits;
+    }
+
+    do {
+      int ord = toGlobal == null ? segOrd : (int) toGlobal.get(segOrd);
+      bits.set(ord);
+      segOrd = (int) subDv.nextOrd();
+    } while (segOrd >= 0);
+  }
+}
+
+
+
+class UniqueMultivaluedSlotAcc extends UniqueSlotAcc implements UnInvertedField.Callback {
+  private UnInvertedField uif;
+  private UnInvertedField.DocToTerm docToTerm;
+
+  public UniqueMultivaluedSlotAcc(FacetContext fcontext, String field, int numSlots, HLLAgg.HLLFactory factory) throws IOException {
+    super(fcontext, field, numSlots, factory);
+    SolrIndexSearcher searcher = fcontext.qcontext.searcher();
+    uif = UnInvertedField.getUnInvertedField(field, searcher);
+    docToTerm = uif.new DocToTerm();
+    fcontext.qcontext.addCloseHook(this);  // TODO: find way to close accumulators instead of using close hook?
+    nTerms = uif.numTerms();
+  }
+
+  @Override
+  protected BytesRef lookupOrd(int ord) throws IOException {
+    return docToTerm.lookupOrd(ord);
+  }
+
+  private FixedBitSet bits;  // bits for the current slot, only set for the callback
+
+  @Override
+  public void call(int termNum) {
+    bits.set(termNum);
+  }
+
+  @Override
+  public void collect(int doc, int slotNum) throws IOException {
+    bits = arr[slotNum];
+    if (bits == null) {
+      bits = new FixedBitSet(nTerms);
+      arr[slotNum] = bits;
+    }
+    docToTerm.getTerms(doc + currentDocBase, this);  // this will call back to our Callback.call(int termNum)
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (docToTerm != null) {
+      docToTerm.close();
+      docToTerm = null;
+    }
+  }
+}
\ No newline at end of file

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java?rev=1679876&r1=1679875&r2=1679876&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java Sun May 17 16:07:33 2015
@@ -28,6 +28,7 @@ import java.util.Map;
 import java.util.Random;
 
 import com.tdunning.math.stats.AVLTreeDigest;
+import net.agkn.hll.HLL;
 import org.apache.lucene.queryparser.flexible.standard.processors.NumericQueryNodeProcessor;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.packed.GrowableWriter;
@@ -294,6 +295,14 @@ public class TestJsonFacets extends Solr
     doStats(Client.localClient, params());
   }
 
+  @Test
+  public void testDistrib() throws Exception {
+    initServers();
+    Client client = servers.getClient(random().nextInt());
+    client.queryDefaults().set( "shards", servers.getShards() );
+    doStats( client, params() );
+  }
+
   public void doStats(Client client, ModifiableSolrParams p) throws Exception {
 
     Map<String, List<String>> fieldLists = new HashMap<>();
@@ -373,8 +382,7 @@ public class TestJsonFacets extends Solr
     client.commit();
 
 
-
-        // straight query facets
+    // straight query facets
     client.testJQ(params(p, "q", "*:*"
             , "json.facet", "{catA:{query:{q:'${cat_s}:A'}},  catA2:{query:{query:'${cat_s}:A'}},  catA3:{query:'${cat_s}:A'}    }"
         )
@@ -460,12 +468,14 @@ public class TestJsonFacets extends Solr
             , "json.facet", "{f1:{type:terms, field:'${cat_s}', sort:'x desc', facet:{x:'min(${num_d})'}  }" +
                 " , f2:{type:terms, field:'${cat_s}', sort:'x desc', facet:{x:'max(${num_d})'}  } " +
                 " , f3:{type:terms, field:'${cat_s}', sort:'x desc', facet:{x:'unique(${where_s})'}  } " +
+                " , f4:{type:terms, field:'${cat_s}', sort:'x desc', facet:{x:'hll(${where_s})'}  } " +
                 "}"
         )
         , "facets=={ 'count':6, " +
             "  f1:{  'buckets':[{ val:'A', count:2, x:2.0 },  { val:'B', count:3, x:-9.0}]}" +
             ", f2:{  'buckets':[{ val:'B', count:3, x:11.0 }, { val:'A', count:2, x:4.0 }]} " +
             ", f3:{  'buckets':[{ val:'A', count:2, x:2 },    { val:'B', count:3, x:2 }]} " +
+            ", f4:{  'buckets':[{ val:'A', count:2, x:2 },    { val:'B', count:3, x:2 }]} " +
             "}"
     );
 
@@ -722,18 +732,28 @@ public class TestJsonFacets extends Solr
 
     // stats at top level
     client.testJQ(params(p, "q", "*:*"
-            , "json.facet", "{ sum1:'sum(${num_d})', sumsq1:'sumsq(${num_d})', avg1:'avg(${num_d})', min1:'min(${num_d})', max1:'max(${num_d})', numwhere:'unique(${where_s})', unique_num_i:'unique(${num_i})', unique_num_d:'unique(${num_d})', unique_date:'unique(${date})',  med:'percentile(${num_d},50)', perc:'percentile(${num_d},0,50.0,100)' }"
+            , "json.facet", "{ sum1:'sum(${num_d})', sumsq1:'sumsq(${num_d})', avg1:'avg(${num_d})', min1:'min(${num_d})', max1:'max(${num_d})'" +
+                ", numwhere:'unique(${where_s})', unique_num_i:'unique(${num_i})', unique_num_d:'unique(${num_d})', unique_date:'unique(${date})'" +
+                ", where_hll:'hll(${where_s})', hll_num_i:'hll(${num_i})', hll_num_d:'hll(${num_d})', hll_date:'hll(${date})'" +
+                ", med:'percentile(${num_d},50)', perc:'percentile(${num_d},0,50.0,100)' }"
         )
         , "facets=={ 'count':6, " +
-            "sum1:3.0, sumsq1:247.0, avg1:0.5, min1:-9.0, max1:11.0, numwhere:2, unique_num_i:4, unique_num_d:5, unique_date:5, med:2.0, perc:[-9.0,2.0,11.0]  }"
+            "sum1:3.0, sumsq1:247.0, avg1:0.5, min1:-9.0, max1:11.0" +
+            ", numwhere:2, unique_num_i:4, unique_num_d:5, unique_date:5" +
+            ", where_hll:2, hll_num_i:4, hll_num_d:5, hll_date:5" +
+            ", med:2.0, perc:[-9.0,2.0,11.0]  }"
     );
 
     // stats at top level, no matches
     client.testJQ(params(p, "q", "id:DOESNOTEXIST"
-            , "json.facet", "{ sum1:'sum(${num_d})', sumsq1:'sumsq(${num_d})', avg1:'avg(${num_d})', min1:'min(${num_d})', max1:'max(${num_d})', numwhere:'unique(${where_s})', unique_num_i:'unique(${num_i})', unique_num_d:'unique(${num_d})', unique_date:'unique(${date})',  med:'percentile(${num_d},50)', perc:'percentile(${num_d},0,50.0,100)' }"
+            , "json.facet", "{ sum1:'sum(${num_d})', sumsq1:'sumsq(${num_d})', avg1:'avg(${num_d})', min1:'min(${num_d})', max1:'max(${num_d})'" +
+                ", numwhere:'unique(${where_s})', unique_num_i:'unique(${num_i})', unique_num_d:'unique(${num_d})', unique_date:'unique(${date})'" +
+                ", where_hll:'hll(${where_s})', hll_num_i:'hll(${num_i})', hll_num_d:'hll(${num_d})', hll_date:'hll(${date})'" +
+                ", med:'percentile(${num_d},50)', perc:'percentile(${num_d},0,50.0,100)' }"
         )
         , "facets=={count:0 " +
-            "/* ,sum1:0.0, sumsq1:0.0, avg1:0.0, min1:'NaN', max1:'NaN', numwhere:0 */ }"
+            "/* ,sum1:0.0, sumsq1:0.0, avg1:0.0, min1:'NaN', max1:'NaN', numwhere:0 */" +
+            " }"
     );
 
     //
@@ -750,11 +770,19 @@ public class TestJsonFacets extends Solr
 
     // test unique on multi-valued field
     client.testJQ(params(p, "q", "*:*"
-            , "json.facet", "{x:'unique(${multi_ss})', y:{query:{q:'id:2', facet:{x:'unique(${multi_ss})'} }}   }"
+            , "json.facet", "{" +
+                "x:'unique(${multi_ss})'" +
+                ",y:{query:{q:'id:2', facet:{x:'unique(${multi_ss})'} }}  " +
+                ",x2:'hll(${multi_ss})'" +
+                ",y2:{query:{q:'id:2', facet:{x:'hll(${multi_ss})'} }}  " +
+
+                " }"
         )
-        , "facets=={ 'count':6, " +
-            "x:2," +
-            "y:{count:1, x:2}" +  // single document should yield 2 unique values
+        , "facets=={count:6 " +
+            ",x:2" +
+            ",y:{count:1, x:2}" +  // single document should yield 2 unique values
+            ",x2:2" +
+            ",y2:{count:1, x:2}" +  // single document should yield 2 unique values
             " }"
     );
 
@@ -936,13 +964,7 @@ public class TestJsonFacets extends Solr
 
   }
 
-  @Test
-  public void testDistrib() throws Exception {
-    initServers();
-    Client client = servers.getClient(random().nextInt());
-    client.queryDefaults().set( "shards", servers.getShards() );
-    doStats( client, params() );
-  }
+
 
 
   @Test
@@ -1002,6 +1024,17 @@ public class TestJsonFacets extends Solr
         , "facets=={ 'count':" + ndocs + "," +
             "'f1':{  'buckets':[{ 'val':'0', 'count':" + ndocs + ", x:" + sz + " }]} } "
     );
+
+    if (client.local()) {
+      // distrib estimation prob won't match
+      client.testJQ(params(p, "q", "*:*"
+              , "json.facet", "{f1:{type:terms, field:${cat_s}, limit:2, facet:{x:'hll($where_s)'}  }}"
+          )
+          , "facets=={ 'count':" + ndocs + "," +
+              "'f1':{  'buckets':[{ 'val':'0', 'count':" + ndocs + ", x:" + sz + " }]} } "
+      );
+    }
+
   }
 
 
@@ -1086,4 +1119,12 @@ public class TestJsonFacets extends Solr
     System.out.println(top.quantile(0.5));
     System.out.println(top.quantile(0.9));
   }
+
+  public void XtestHLL() {
+    HLLAgg.HLLFactory fac = new HLLAgg.HLLFactory();
+    HLL hll = fac.getHLL();
+    hll.addRaw(123456789);
+    hll.addRaw(987654321);
+  }
+
 }

Modified: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/util/Hash.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/util/Hash.java?rev=1679876&r1=1679875&r2=1679876&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/util/Hash.java (original)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/util/Hash.java Sun May 17 16:07:33 2015
@@ -419,4 +419,102 @@ public class Hash {
   }
 
 
+  /** 128 bits of state */
+  public static final class LongPair {
+    public long val1;
+    public long val2;
+  }
+
+  public static final int fmix32(int h) {
+    h ^= h >>> 16;
+    h *= 0x85ebca6b;
+    h ^= h >>> 13;
+    h *= 0xc2b2ae35;
+    h ^= h >>> 16;
+    return h;
+  }
+
+  public static final long fmix64(long k) {
+    k ^= k >>> 33;
+    k *= 0xff51afd7ed558ccdL;
+    k ^= k >>> 33;
+    k *= 0xc4ceb9fe1a85ec53L;
+    k ^= k >>> 33;
+    return k;
+  }
+
+  /** Gets a long from a byte buffer in little endian byte order. */
+  public static final long getLongLittleEndian(byte[] buf, int offset) {
+    return     ((long)buf[offset+7]    << 56)   // no mask needed
+        | ((buf[offset+6] & 0xffL) << 48)
+        | ((buf[offset+5] & 0xffL) << 40)
+        | ((buf[offset+4] & 0xffL) << 32)
+        | ((buf[offset+3] & 0xffL) << 24)
+        | ((buf[offset+2] & 0xffL) << 16)
+        | ((buf[offset+1] & 0xffL) << 8)
+        | ((buf[offset  ] & 0xffL));        // no shift needed
+  }
+
+
+  /** Returns the MurmurHash3_x64_128 hash, placing the result in "out". */
+  public static void murmurhash3_x64_128(byte[] key, int offset, int len, int seed, LongPair out) {
+    // The original algorithm does have a 32 bit unsigned seed.
+    // We have to mask to match the behavior of the unsigned types and prevent sign extension.
+    long h1 = seed & 0x00000000FFFFFFFFL;
+    long h2 = seed & 0x00000000FFFFFFFFL;
+
+    final long c1 = 0x87c37b91114253d5L;
+    final long c2 = 0x4cf5ad432745937fL;
+
+    int roundedEnd = offset + (len & 0xFFFFFFF0);  // round down to 16 byte block
+    for (int i=offset; i<roundedEnd; i+=16) {
+      long k1 = getLongLittleEndian(key, i);
+      long k2 = getLongLittleEndian(key, i+8);
+      k1 *= c1; k1  = Long.rotateLeft(k1,31); k1 *= c2; h1 ^= k1;
+      h1 = Long.rotateLeft(h1,27); h1 += h2; h1 = h1*5+0x52dce729;
+      k2 *= c2; k2  = Long.rotateLeft(k2,33); k2 *= c1; h2 ^= k2;
+      h2 = Long.rotateLeft(h2,31); h2 += h1; h2 = h2*5+0x38495ab5;
+    }
+
+    long k1 = 0;
+    long k2 = 0;
+
+    switch (len & 15) {
+      case 15: k2  = (key[roundedEnd+14] & 0xffL) << 48;
+      case 14: k2 |= (key[roundedEnd+13] & 0xffL) << 40;
+      case 13: k2 |= (key[roundedEnd+12] & 0xffL) << 32;
+      case 12: k2 |= (key[roundedEnd+11] & 0xffL) << 24;
+      case 11: k2 |= (key[roundedEnd+10] & 0xffL) << 16;
+      case 10: k2 |= (key[roundedEnd+ 9] & 0xffL) << 8;
+      case  9: k2 |= (key[roundedEnd+ 8] & 0xffL);
+        k2 *= c2; k2  = Long.rotateLeft(k2, 33); k2 *= c1; h2 ^= k2;
+      case  8: k1  = ((long)key[roundedEnd+7]) << 56;
+      case  7: k1 |= (key[roundedEnd+6] & 0xffL) << 48;
+      case  6: k1 |= (key[roundedEnd+5] & 0xffL) << 40;
+      case  5: k1 |= (key[roundedEnd+4] & 0xffL) << 32;
+      case  4: k1 |= (key[roundedEnd+3] & 0xffL) << 24;
+      case  3: k1 |= (key[roundedEnd+2] & 0xffL) << 16;
+      case  2: k1 |= (key[roundedEnd+1] & 0xffL) << 8;
+      case  1: k1 |= (key[roundedEnd  ] & 0xffL);
+        k1 *= c1; k1  = Long.rotateLeft(k1,31); k1 *= c2; h1 ^= k1;
+    }
+
+    //----------
+    // finalization
+
+    h1 ^= len; h2 ^= len;
+
+    h1 += h2;
+    h2 += h1;
+
+    h1 = fmix64(h1);
+    h2 = fmix64(h2);
+
+    h1 += h2;
+    h2 += h1;
+
+    out.val1 = h1;
+    out.val2 = h2;
+  }
+
 }