You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ho...@apache.org on 2018/05/21 16:35:43 UTC

[2/2] lucene-solr:branch_7x: SOLR-9480: A new 'relatedness()' aggregate function for JSON Faceting to enable building Semantic Knowledge Graphs

SOLR-9480: A new 'relatedness()' aggregate function for JSON Faceting to enable building Semantic Knowledge Graphs

(cherry picked from commit 669b9e7a5343c625e265a075c9dbf24fcbff7363)


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/f0d6a0e6
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/f0d6a0e6
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/f0d6a0e6

Branch: refs/heads/branch_7x
Commit: f0d6a0e638b13ddf4f5acfffdcd390e977572b67
Parents: a69321a
Author: Chris Hostetter <ho...@apache.org>
Authored: Mon May 21 08:22:54 2018 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Mon May 21 08:26:57 2018 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../apache/solr/search/ValueSourceParser.java   |  12 +
 .../apache/solr/search/facet/FacetField.java    |   2 +-
 .../solr/search/facet/FacetFieldProcessor.java  |  43 +-
 .../facet/FacetFieldProcessorByArray.java       |  25 +-
 .../facet/FacetFieldProcessorByArrayDV.java     |   4 +-
 .../FacetFieldProcessorByEnumTermsStream.java   |  17 +-
 .../facet/FacetFieldProcessorByHashDV.java      |  10 +-
 .../apache/solr/search/facet/FacetMerger.java   |   6 +-
 .../solr/search/facet/FacetProcessor.java       |  61 +-
 .../apache/solr/search/facet/FacetRange.java    |   3 +-
 .../apache/solr/search/facet/FacetRequest.java  |  53 +-
 .../search/facet/FacetRequestSortedMerger.java  |   2 +-
 .../org/apache/solr/search/facet/HLLAgg.java    |   3 +-
 .../org/apache/solr/search/facet/MinMaxAgg.java |   9 +-
 .../apache/solr/search/facet/PercentileAgg.java |   3 +-
 .../solr/search/facet/RelatednessAgg.java       | 449 +++++++++++++
 .../org/apache/solr/search/facet/SlotAcc.java   |  74 ++-
 .../solr/search/facet/UnInvertedField.java      |   8 +-
 .../org/apache/solr/search/facet/UniqueAgg.java |   3 +-
 .../solr/search/facet/UniqueMultiDvSlotAcc.java |   3 +-
 .../search/facet/UniqueMultivaluedSlotAcc.java  |   5 +-
 .../search/facet/UniqueSinglevaluedSlotAcc.java |   3 +-
 .../solr/cloud/TestCloudJSONFacetSKG.java       | 654 +++++++++++++++++++
 .../apache/solr/search/QueryEqualityTest.java   |  10 +
 .../org/apache/solr/search/facet/DebugAgg.java  |   9 +-
 ...ibutedFacetSimpleRefinementLongTailTest.java |  19 +-
 .../search/facet/TestJsonFacetRefinement.java   |  69 ++
 .../solr/search/facet/TestJsonFacets.java       | 239 +++++++
 solr/solr-ref-guide/src/json-facet-api.adoc     | 144 +++-
 30 files changed, 1848 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index f60006e..574e22c 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -87,6 +87,9 @@ New Features
 
 * SOLR-11277: Add auto hard-commit settings based on tlog size (Rupa Shankar, Anshum Gupta)
 
+* SOLR-9480: A new 'relatedness()' aggregate function for JSON Faceting to enable building Semantic
+  Knowledge Graphs. (Trey Grainger, hossman)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java b/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
index 683cf4a..b7c6815 100644
--- a/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
+++ b/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
@@ -63,6 +63,7 @@ import org.apache.solr.search.facet.PercentileAgg;
 import org.apache.solr.search.facet.StddevAgg;
 import org.apache.solr.search.facet.SumAgg;
 import org.apache.solr.search.facet.SumsqAgg;
+import org.apache.solr.search.facet.RelatednessAgg;
 import org.apache.solr.search.facet.UniqueAgg;
 import org.apache.solr.search.facet.UniqueBlockAgg;
 import org.apache.solr.search.facet.VarianceAgg;
@@ -1039,6 +1040,17 @@ public abstract class ValueSourceParser implements NamedListInitializedPlugin {
 
     addParser("agg_percentile", new PercentileAgg.Parser());
     
+    addParser("agg_" + RelatednessAgg.NAME, new ValueSourceParser() {
+      @Override
+      public ValueSource parse(FunctionQParser fp) throws SyntaxError {
+        // TODO: (fore & back)-ground should be optional -- use hasMoreArguments
+        // if only one arg, assume it's the foreground
+        // (background is the one that will most commonly just be "*:*")
+        // see notes in RelatednessAgg constructor about why we don't do this yet
+        return new RelatednessAgg(fp.parseNestedQuery(), fp.parseNestedQuery());
+      }
+    });
+    
     addParser("childfield", new ChildFieldValueSourceParser());
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/FacetField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetField.java b/solr/core/src/java/org/apache/solr/search/facet/FacetField.java
index 1a59549..4de0411 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetField.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetField.java
@@ -42,7 +42,7 @@ abstract class FacetRequestSorted extends FacetRequest {
 
   @Override
   public boolean returnsPartial() {
-    return limit > 0;
+    return super.returnsPartial() || (limit > 0);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java
index f872db3..7276593 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java
@@ -36,6 +36,7 @@ import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.DocSet;
+import org.apache.solr.search.facet.SlotAcc.SlotContext;
 
 import static org.apache.solr.search.facet.FacetContext.SKIP_FACET;
 
@@ -226,23 +227,23 @@ abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
     }
   }
 
-  int collectFirstPhase(DocSet docs, int slot) throws IOException {
+  int collectFirstPhase(DocSet docs, int slot, IntFunction<SlotContext> slotContext) throws IOException {
     int num = -1;
     if (collectAcc != null) {
-      num = collectAcc.collect(docs, slot);
+      num = collectAcc.collect(docs, slot, slotContext);
     }
     if (allBucketsAcc != null) {
-      num = allBucketsAcc.collect(docs, slot);
+      num = allBucketsAcc.collect(docs, slot, slotContext);
     }
     return num >= 0 ? num : docs.size();
   }
 
-  void collectFirstPhase(int segDoc, int slot) throws IOException {
+  void collectFirstPhase(int segDoc, int slot, IntFunction<SlotContext> slotContext) throws IOException {
     if (collectAcc != null) {
-      collectAcc.collect(segDoc, slot);
+      collectAcc.collect(segDoc, slot, slotContext);
     }
     if (allBucketsAcc != null) {
-      allBucketsAcc.collect(segDoc, slot);
+      allBucketsAcc.collect(segDoc, slot, slotContext);
     }
   }
 
@@ -373,7 +374,7 @@ abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
       Comparable val = bucketValFromSlotNumFunc.apply(slotNum);
       bucket.add("val", val);
 
-      Query filter = needFilter ? sf.getType().getFieldQuery(null, sf, fieldQueryValFunc.apply(val)) : null;
+      Query filter = needFilter ? makeBucketQuery(fieldQueryValFunc.apply(val)) : null;
 
       fillBucket(bucket, countAcc.getCount(slotNum), slotNum, null, filter);
 
@@ -388,6 +389,15 @@ abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
     return res;
   }
 
+  /**
+   * Trivial helper method for building up a bucket query given the (Stringified) bucket value
+   */
+  protected Query makeBucketQuery(final String bucketValue) {
+    // TODO: this isn't viable for things like text fields w/ analyzers that are non-idempotent (ie: stemmers)
+    // TODO: but changing it to just use TermQuery isn't safe for things like numerics, dates, etc...
+    return sf.getType().getFieldQuery(null, sf, bucketValue);
+  }
+
   private void calculateNumBuckets(SimpleOrderedMap<Object> target) throws IOException {
     DocSet domain = fcontext.base;
     if (freq.prefix != null) {
@@ -397,7 +407,7 @@ abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
 
     HLLAgg agg = new HLLAgg(freq.field);
     SlotAcc acc = agg.createSlotAcc(fcontext, domain.size(), 1);
-    acc.collect(domain, 0);
+    acc.collect(domain, 0, null); // we know HLL doesn't care about the bucket query
     acc.key = "numBuckets";
     acc.setValues(target, 0);
   }
@@ -433,7 +443,7 @@ abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
       // 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.collect(subDomain, 0, slot -> { return new SlotContext(filter); });
         acc.setValues(target, 0);
       }
     }
@@ -442,13 +452,14 @@ abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
   }
 
   @Override
-  protected void processStats(SimpleOrderedMap<Object> bucket, DocSet docs, int docCount) throws IOException {
+  protected void processStats(SimpleOrderedMap<Object> bucket, Query bucketQ, 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);
+    assert null != bucketQ;
+    int collected = collect(docs, 0, slotNum -> { return new SlotContext(bucketQ); });
 
     // countAcc.incrementCount(0, collected);  // should we set the counton the acc instead of just passing it?
 
@@ -499,9 +510,9 @@ abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
     }
 
     @Override
-    public void collect(int doc, int slot) throws IOException {
+    public void collect(int doc, int slot, IntFunction<SlotContext> slotContext) throws IOException {
       for (SlotAcc acc : subAccs) {
-        acc.collect(doc, slot);
+        acc.collect(doc, slot, slotContext);
       }
     }
 
@@ -561,15 +572,15 @@ abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
     }
 
     @Override
-    public void collect(int doc, int slot) throws IOException {
+    public void collect(int doc, int slot, IntFunction<SlotContext> slotContext) throws IOException {
       assert slot != collectAccSlot || slot < 0;
       count++;
       if (collectAcc != null) {
-        collectAcc.collect(doc, collectAccSlot);
+        collectAcc.collect(doc, collectAccSlot, slotContext);
       }
       if (otherAccs != null) {
         for (SlotAcc otherAcc : otherAccs) {
-          otherAcc.collect(doc, otherAccsSlot);
+          otherAcc.collect(doc, otherAccsSlot, slotContext);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java
index 43bafd4..e5ee181 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java
@@ -19,11 +19,14 @@ package org.apache.solr.search.facet;
 
 import java.io.IOException;
 import java.util.Date;
+import java.util.function.IntFunction;
 
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.search.Query;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.facet.SlotAcc.SlotContext;
 
 import static org.apache.solr.search.facet.FacetContext.SKIP_FACET;
 
@@ -116,8 +119,28 @@ abstract class FacetFieldProcessorByArray extends FacetFieldProcessor {
             throw new RuntimeException(e);
           }
         },
-        obj -> obj instanceof Date ? ((Date)obj).toInstant().toString() : obj.toString()
+        obj -> valueObjToString(obj)
     );
   }
 
+  private static String valueObjToString(Object obj) {
+    return (obj instanceof Date) ? ((Date)obj).toInstant().toString() : obj.toString();
+  }
+
+                                                           
+  /**
+   * SlotContext to use during all {@link SlotAcc} collection.
+   *
+   * @see #lookupOrd
+   */
+  public IntFunction<SlotContext> slotContext = (slotNum) -> {
+    try {
+      Object value = sf.getType().toObject(sf, lookupOrd(slotNum));
+      Query q = makeBucketQuery(valueObjToString(value));
+      assert null != q : "null query for: '" + value + "'";
+      return new SlotContext(q);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  };
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArrayDV.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArrayDV.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArrayDV.java
index fe7a3f2..4a0d13e 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArrayDV.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArrayDV.java
@@ -327,10 +327,10 @@ class FacetFieldProcessorByArrayDV extends FacetFieldProcessorByArray {
     if (arrIdx >= 0 && arrIdx < nTerms) {
       countAcc.incrementCount(arrIdx, 1);
       if (collectAcc != null) {
-        collectAcc.collect(doc, arrIdx);
+        collectAcc.collect(doc, arrIdx, slotContext);
       }
       if (allBucketsAcc != null) {
-        allBucketsAcc.collect(doc, arrIdx);
+        allBucketsAcc.collect(doc, arrIdx, slotContext);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByEnumTermsStream.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByEnumTermsStream.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByEnumTermsStream.java
index bbc2973..f939bba 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByEnumTermsStream.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByEnumTermsStream.java
@@ -21,6 +21,7 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.util.Iterator;
 import java.util.List;
+import java.util.function.IntFunction;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.MultiPostingsEnum;
@@ -40,6 +41,7 @@ import org.apache.solr.search.DocSet;
 import org.apache.solr.search.HashDocSet;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.search.SortedIntDocSet;
+import org.apache.solr.search.facet.SlotAcc.SlotContext;
 
 /**
  * Enumerates indexed terms in order in a streaming fashion.
@@ -60,6 +62,13 @@ class FacetFieldProcessorByEnumTermsStream extends FacetFieldProcessor implement
   PostingsEnum postingsEnum;
   BytesRef startTermBytes;
   BytesRef term;
+
+  // at any point in processing where we need a SlotContext, all we care about is the current 'term'
+  private IntFunction<SlotContext> slotContext = (slotNum) -> {
+    assert null != term;
+    return new SlotAcc.SlotContext(new TermQuery(new Term(sf.getName(), term)));
+  };
+  
   LeafReaderContext[] leaves;
 
   FacetFieldProcessorByEnumTermsStream(FacetContext fcontext, FacetField freq, SchemaField sf) {
@@ -195,7 +204,7 @@ class FacetFieldProcessorByEnumTermsStream extends FacetFieldProcessor implement
 
   private SimpleOrderedMap<Object> _nextBucket() throws IOException {
     DocSet termSet = null;
-
+    
     try {
       while (term != null) {
 
@@ -241,7 +250,7 @@ class FacetFieldProcessorByEnumTermsStream extends FacetFieldProcessor implement
             resetStats();
 
             if (!countOnly) {
-              collect(termSet, 0);
+              collect(termSet, 0, slotContext);
             }
 
         } else {
@@ -278,7 +287,7 @@ class FacetFieldProcessorByEnumTermsStream extends FacetFieldProcessor implement
                 while ((docid = sub.postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
                   if (fastForRandomSet.exists(docid + base)) {
                     c++;
-                    collect(docid, 0);
+                    collect(docid, 0, slotContext);
                   }
                 }
               }
@@ -295,7 +304,7 @@ class FacetFieldProcessorByEnumTermsStream extends FacetFieldProcessor implement
               while ((docid = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
                 if (fastForRandomSet.exists(docid)) {
                   c++;
-                  collect(docid, 0);
+                  collect(docid, 0, slotContext);
                 }
               }
             }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByHashDV.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByHashDV.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByHashDV.java
index a335c51..441780c 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByHashDV.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByHashDV.java
@@ -36,6 +36,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.DocSetUtil;
+import org.apache.solr.search.facet.SlotAcc.SlotContext;
 
 /**
  * Facets numbers into a hash table.  The number is either a raw numeric DocValues value, or
@@ -259,7 +260,7 @@ class FacetFieldProcessorByHashDV extends FacetFieldProcessor {
 
     indexOrderAcc = new SlotAcc(fcontext) {
       @Override
-      public void collect(int doc, int slot) throws IOException {
+      public void collect(int doc, int slot, IntFunction<SlotContext> slotContext) throws IOException {
       }
 
       @Override
@@ -305,7 +306,7 @@ class FacetFieldProcessorByHashDV extends FacetFieldProcessor {
       }
 
       @Override
-      public void collect(int doc, int slot) throws IOException {
+      public void collect(int doc, int slot, IntFunction<SlotContext> slotContext) throws IOException {
         throw new UnsupportedOperationException();
       }
 
@@ -428,7 +429,10 @@ class FacetFieldProcessorByHashDV extends FacetFieldProcessor {
     // Our countAcc is virtual, so this is not needed:
     // countAcc.incrementCount(slot, 1);
 
-    super.collectFirstPhase(segDoc, slot);
+    super.collectFirstPhase(segDoc, slot, slotNum -> {
+        Comparable value = calc.bitsToValue(val);
+        return new SlotContext(sf.getType().getFieldQuery(null, sf, calc.formatValue(value)));
+      });
   }
 
   private void doRehash(LongCounts table) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/FacetMerger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetMerger.java b/solr/core/src/java/org/apache/solr/search/facet/FacetMerger.java
index 9499d2c..84ffbf0 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetMerger.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetMerger.java
@@ -120,8 +120,10 @@ public abstract class FacetMerger {
 
       subs = null;
       for (Map.Entry<String,FacetRequest> entry : freq.subFacets.entrySet()) {
-        Collection<String> childSubs = getSubsWithPartial(entry.getValue());
-        if (childSubs.size() > 0 || entry.getValue().returnsPartial()) {
+        final FacetRequest entryVal = entry.getValue();
+        Collection<String> childSubs = getSubsWithPartial(entryVal);
+        // TODO: should returnsPartial() check processEmpty internally?
+        if (childSubs.size() > 0 || entryVal.returnsPartial() || entryVal.processEmpty) {
           if (subs == null) {
             subs = new ArrayList<>(freq.getSubFacets().size());
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java b/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java
index 27cdaec..c0625df 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java
@@ -24,6 +24,7 @@ import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.function.IntFunction;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.BooleanClause;
@@ -42,6 +43,7 @@ import org.apache.solr.search.QParser;
 import org.apache.solr.search.QueryContext;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.search.SyntaxError;
+import org.apache.solr.search.facet.SlotAcc.SlotContext;
 import org.apache.solr.util.RTimer;
 
 public abstract class FacetProcessor<FacetRequestT extends FacetRequest>  {
@@ -78,6 +80,7 @@ public abstract class FacetProcessor<FacetRequestT extends FacetRequest>  {
   FacetProcessor(FacetContext fcontext, FacetRequestT freq) {
     this.fcontext = fcontext;
     this.freq = freq;
+    fcontext.processor = this;
   }
 
   public Object getResponse() {
@@ -90,10 +93,13 @@ public abstract class FacetProcessor<FacetRequestT extends FacetRequest>  {
 
   private void evalFilters() throws IOException {
     if (freq.domain.filters == null || freq.domain.filters.isEmpty()) return;
-
-    List<Query> qlist = new ArrayList<>(freq.domain.filters.size());
+    this.filter = fcontext.searcher.getDocSet(evalJSONFilterQueryStruct(fcontext, freq.domain.filters));
+  }
+  
+  private static List<Query> evalJSONFilterQueryStruct(FacetContext fcontext, List<Object> filters) throws IOException {
+    List<Query> qlist = new ArrayList<>(filters.size());
     // TODO: prevent parsing filters each time!
-    for (Object rawFilter : freq.domain.filters) {
+    for (Object rawFilter : filters) {
       if (rawFilter instanceof String) {
         QParser parser = null;
         try {
@@ -149,13 +155,30 @@ public abstract class FacetProcessor<FacetRequestT extends FacetRequest>  {
       }
 
     }
-
-    this.filter = fcontext.searcher.getDocSet(qlist);
+    return qlist;
   }
 
   private void handleDomainChanges() throws IOException {
     if (freq.domain == null) return;
-    handleFilterExclusions();
+
+    if (null != freq.domain.explicitQueries) {
+      try {
+        final List<Query> domainQs = evalJSONFilterQueryStruct(fcontext, freq.domain.explicitQueries);
+        if (domainQs.isEmpty()) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+                                  "'query' domain must not evaluate to an empty list of queries");
+        }
+        fcontext.base = fcontext.searcher.getDocSet(domainQs);
+      } catch (SolrException e) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+                                "Unable to parse domain 'query': " + freq.domain.explicitQueries +
+                                " -- reason: " + e.getMessage(),
+                                e);
+      }
+    } else {
+      // mutualy exclusive to freq.domain.explicitQueries
+      handleFilterExclusions();
+    }
 
     // Check filters... if we do have filters they apply after domain changes.
     // We still calculate them first because we can use it in a parent->child domain change.
@@ -277,13 +300,13 @@ public abstract class FacetProcessor<FacetRequestT extends FacetRequest>  {
     return appliedFilters;
   }
 
-  protected void processStats(SimpleOrderedMap<Object> bucket, DocSet docs, int docCount) throws IOException {
+  protected void processStats(SimpleOrderedMap<Object> bucket, Query bucketQ, 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);
+    int collected = collect(docs, 0, slotNum -> { return new SlotContext(bucketQ); });
     countAcc.incrementCount(0, collected);
     assert collected == docCount;
     addStats(bucket, 0);
@@ -319,10 +342,22 @@ public abstract class FacetProcessor<FacetRequestT extends FacetRequest>  {
     }
   }
 
-  int collect(DocSet docs, int slot) throws IOException {
+  int collect(DocSet docs, int slot, IntFunction<SlotContext> slotContext) throws IOException {
     int count = 0;
     SolrIndexSearcher searcher = fcontext.searcher;
 
+    if (0 == docs.size()) {
+      // we may be in a "processEmpty" type situation where the client still cares about this bucket
+      // either way, we should let our accumulators know about the empty set, so they can collect &
+      // compute the slot (ie: let them decide if they care even when it's size==0)
+      if (accs != null) {
+        for (SlotAcc acc : accs) {
+          acc.collect(docs, slot, slotContext); // NOT per-seg collectors
+        }
+      }
+      return count;
+    }
+    
     final List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
     final Iterator<LeafReaderContext> ctxIt = leaves.iterator();
     LeafReaderContext ctx = null;
@@ -346,15 +381,15 @@ public abstract class FacetProcessor<FacetRequestT extends FacetRequest>  {
         setNextReader(ctx);
       }
       count++;
-      collect(doc - segBase, slot);  // per-seg collectors
+      collect(doc - segBase, slot, slotContext);  // per-seg collectors
     }
     return count;
   }
 
-  void collect(int segDoc, int slot) throws IOException {
+  void collect(int segDoc, int slot, IntFunction<SlotContext> slotContext) throws IOException {
     if (accs != null) {
       for (SlotAcc acc : accs) {
-        acc.collect(segDoc, slot);
+        acc.collect(segDoc, slot, slotContext);
       }
     }
   }
@@ -402,7 +437,7 @@ public abstract class FacetProcessor<FacetRequestT extends FacetRequest>  {
 
     try {
       if (!skip) {
-        processStats(bucket, result, count);
+        processStats(bucket, q, result, count);
       }
       processSubs(bucket, q, result, skip, facetInfo);
     } finally {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java b/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java
index 09b8ec0..817ada6 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java
@@ -38,6 +38,7 @@ import org.apache.solr.schema.SchemaField;
 import org.apache.solr.schema.TrieDateField;
 import org.apache.solr.schema.TrieField;
 import org.apache.solr.search.DocSet;
+import org.apache.solr.search.facet.SlotAcc.SlotContext;
 import org.apache.solr.util.DateMathParser;
 
 import static org.apache.solr.search.facet.FacetContext.SKIP_FACET;
@@ -364,7 +365,7 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
     DocSet intersection = fcontext.searcher.getDocSet(rangeQ, fcontext.base);
     filters[slot] = rangeQ;
     intersections[slot] = intersection;  // save for later  // TODO: only save if number of slots is small enough?
-    int num = collect(intersection, slot);
+    int num = collect(intersection, slot, slotNum -> { return new SlotContext(rangeQ); });
     countAcc.incrementCount(slot, num); // TODO: roll this into collect()
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java b/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
index 4cf8a68..6337bcb 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
@@ -87,6 +87,15 @@ public abstract class FacetRequest {
 
   // domain changes
   public static class Domain {
+    /** 
+     * An explicit query domain, <em>ignoring all parent context</em>, expressed in JSON query format.
+     * Mutually exclusive to {@link #excludeTags}
+     */
+    public List<Object> explicitQueries; // list of symbolic filters (JSON query format)
+    /**
+     * Specifies query/filter tags that should be excluded to re-compute the domain from the parent context.
+     * Mutually exclusive to {@link #explicitQueries}
+     */
     public List<String> excludeTags;
     public JoinField joinField;
     public boolean toParent;
@@ -96,12 +105,13 @@ public abstract class FacetRequest {
 
     // True if a starting set of documents can be mapped onto a different set of documents not originally in the starting set.
     public boolean canTransformDomain() {
-      return toParent || toChildren || (excludeTags != null) || (joinField != null);
+      return toParent || toChildren
+        || (explicitQueries != null) || (excludeTags != null) || (joinField != null);
     }
 
     // Can this domain become non-empty if the input domain is empty?  This does not check any sub-facets (see canProduceFromEmpty for that)
     public boolean canBecomeNonEmpty() {
-      return excludeTags != null;
+      return (explicitQueries != null) || (excludeTags != null);
     }
 
     /** Are we doing a query time join across other documents */
@@ -197,6 +207,7 @@ public abstract class FacetRequest {
    * This is normally true only for facets with a limit.
    */
   public boolean returnsPartial() {
+    // TODO: should the default impl check processEmpty ?
     return false;
   }
 
@@ -242,6 +253,7 @@ class FacetContext {
   public static final int IS_REFINEMENT=0x02;
   public static final int SKIP_FACET=0x04;  // refinement: skip calculating this immediate facet, but proceed to specific sub-facets based on facetInfo
 
+  FacetProcessor processor;
   Map<String,Object> facetInfo; // refinement info for this node
   QueryContext qcontext;
   SolrQueryRequest req;  // TODO: replace with params?
@@ -459,6 +471,17 @@ abstract class FacetParser<FacetRequestT extends FacetRequest> {
           domain.excludeTags = excludeTags;
         }
 
+        if (domainMap.containsKey("query")) {
+          domain.explicitQueries = parseJSONQueryStruct(domainMap.get("query"));
+          if (null == domain.explicitQueries) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+                                    "'query' domain can not be null or empty");
+          } else if (null != domain.excludeTags) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+                                    "'query' domain can not be combined with 'excludeTags'");
+          }
+        }
+        
         String blockParent = (String)domainMap.get("blockParent");
         String blockChildren = (String)domainMap.get("blockChildren");
 
@@ -475,21 +498,29 @@ abstract class FacetParser<FacetRequestT extends FacetRequest> {
         Object filterOrList = domainMap.get("filter");
         if (filterOrList != null) {
           assert domain.filters == null;
-          if (filterOrList instanceof List) {
-            domain.filters = (List<Object>)filterOrList;
-          } else {
-            domain.filters = new ArrayList<>(1);
-            domain.filters.add(filterOrList);
-          }
+          domain.filters = parseJSONQueryStruct(filterOrList);
         }
 
-
       } // end "domain"
-
-
     }
   }
 
+  /** returns null on null input, otherwise returns a list of the JSON query structures -- either
+   * directly from the raw (list) input, or if raw input is a not a list then it encapsulates 
+   * it in a new list.
+   */
+  private List<Object> parseJSONQueryStruct(Object raw) {
+    List<Object> result = null;
+    if (null == raw) {
+      return result;
+    } else if (raw instanceof List) {
+      result = (List<Object>) raw;
+    } else {
+      result = new ArrayList<>(1);
+      result.add(raw);
+    }
+    return result;
+  }
 
   public String getField(Map<String,Object> args) {
     Object fieldName = args.get("field"); // TODO: pull out into defined constant

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/FacetRequestSortedMerger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetRequestSortedMerger.java b/solr/core/src/java/org/apache/solr/search/facet/FacetRequestSortedMerger.java
index 9ffdea7..d737885 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetRequestSortedMerger.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetRequestSortedMerger.java
@@ -162,7 +162,7 @@ abstract class FacetRequestSortedMerger<FacetRequestT extends FacetRequestSorted
 
     // TODO: add information in sub-shard response about dropped buckets (i.e. not all returned due to limit)
     // If we know we've seen all the buckets from a shard, then we don't have to add to leafBuckets or partialBuckets, only skipBuckets
-    boolean isCommandPartial = freq.returnsPartial();
+    boolean isCommandPartial = freq.returnsPartial() || freq.processEmpty; // TODO: should returnsPartial() check processEmpty internally?
     boolean returnedAllBuckets = !isCommandPartial && !thisMissing;  // did the shard return all of the possible buckets?
 
     if (returnedAllBuckets && tags.isEmpty() && tagsWithPartial.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java b/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java
index 897dceb..4634bc2 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java
@@ -17,6 +17,7 @@
 package org.apache.solr.search.facet;
 
 import java.io.IOException;
+import java.util.function.IntFunction;
 
 import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -140,7 +141,7 @@ public class HLLAgg extends StrAggValueSource {
     }
 
     @Override
-    public void collect(int doc, int slot) throws IOException {
+    public void collect(int doc, int slot, IntFunction<SlotContext> slotContext) throws IOException {
       int valuesDocID = docIdSetIterator().docID();
       if (valuesDocID < doc) {
         valuesDocID = docIdSetIterator().advance(doc);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/MinMaxAgg.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/MinMaxAgg.java b/solr/core/src/java/org/apache/solr/search/facet/MinMaxAgg.java
index 8d4dc4d..1c961e0 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/MinMaxAgg.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/MinMaxAgg.java
@@ -19,6 +19,7 @@ package org.apache.solr.search.facet;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Date;
+import java.util.function.IntFunction;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.MultiDocValues;
@@ -142,7 +143,7 @@ public class MinMaxAgg extends SimpleAggValueSource {
     }
 
     @Override
-    public void collect(int doc, int slotNum) throws IOException {
+    public void collect(int doc, int slotNum, IntFunction<SlotContext> slotContext) throws IOException {
       double val = values.doubleVal(doc);
       if (val == 0 && !values.exists(doc)) return; // depend on fact that non existing values return 0 for func query
 
@@ -171,7 +172,7 @@ public class MinMaxAgg extends SimpleAggValueSource {
     }
 
     @Override
-    public void collect(int doc, int slotNum) throws IOException {
+    public void collect(int doc, int slotNum, IntFunction<SlotContext> slotContext) throws IOException {
       long val = values.longVal(doc);
       if (val == 0 && !values.exists(doc)) return; // depend on fact that non existing values return 0 for func query
 
@@ -230,7 +231,7 @@ public class MinMaxAgg extends SimpleAggValueSource {
     }
 
     @Override
-    public void collect(int doc, int slotNum) throws IOException {
+    public void collect(int doc, int slotNum, IntFunction<SlotContext> slotContext) throws IOException {
       long val = values.longVal(doc);
       if (val == 0 && !values.exists(doc)) return; // depend on fact that non existing values return 0 for func query
 
@@ -334,7 +335,7 @@ public class MinMaxAgg extends SimpleAggValueSource {
     }
 
     @Override
-    public void collect(int doc, int slotNum) throws IOException {
+    public void collect(int doc, int slotNum, IntFunction<SlotContext> slotContext) throws IOException {
       if (subDv.advanceExact(doc)) {
         int segOrd = subDv.ordValue();
         int ord = toGlobal==null ? segOrd : (int)toGlobal.get(segOrd);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/PercentileAgg.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/PercentileAgg.java b/solr/core/src/java/org/apache/solr/search/facet/PercentileAgg.java
index ea46a91..efdef55 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/PercentileAgg.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/PercentileAgg.java
@@ -21,6 +21,7 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.function.IntFunction;
 
 import com.tdunning.math.stats.AVLTreeDigest;
 import org.apache.lucene.queries.function.ValueSource;
@@ -109,7 +110,7 @@ public class PercentileAgg extends SimpleAggValueSource {
       digests = new AVLTreeDigest[numSlots];
     }
 
-    public void collect(int doc, int slotNum) throws IOException {
+    public void collect(int doc, int slotNum, IntFunction<SlotContext> slotContext) throws IOException {
       if (!values.exists(doc)) return;
       double val = values.doubleVal(doc);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/RelatednessAgg.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/RelatednessAgg.java b/solr/core/src/java/org/apache/solr/search/facet/RelatednessAgg.java
new file mode 100644
index 0000000..5bc1108
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/facet/RelatednessAgg.java
@@ -0,0 +1,449 @@
+/*
+ * 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.
+ */
+package org.apache.solr.search.facet;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+import java.util.Map;
+import java.util.function.IntFunction;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.search.Query;
+
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.search.DocSet;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An aggregation function designed to be nested under other (possibly deeply nested) facets for 
+ * the purposes of computing the "relatedness" of facet buckets relative to 
+ * "foreground" and "background" sets -- primarily for the purpose of building "Semantic Knowledge Graphs"
+ *
+ * @see <a href="https://arxiv.org/pdf/1609.00464.pdf">The Semantic Knowledge Graph: 
+ *     A compact, auto-generated model for real-time traversal and ranking of any relationship 
+ *     within a domain</a>
+ */
+public class RelatednessAgg extends AggValueSource {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  // end user values
+  private static final String RELATEDNESS = "relatedness";
+  private static final String FG_POP = "foreground_popularity";
+  private static final String BG_POP = "background_popularity";
+
+  // needed for distrib calculation
+  private static final String FG_SIZE = "foreground_size";
+  private static final String FG_COUNT = "foreground_count";
+  private static final String BG_SIZE = "background_size";
+  private static final String BG_COUNT = "background_count";
+  
+  final protected Query fgQ;
+  final protected Query bgQ;
+  
+  public static final String NAME = RELATEDNESS;
+  public RelatednessAgg(Query fgQ, Query bgQ) {
+    super(NAME); 
+    // NOTE: ideally we don't want to assume any defaults *yet* if fgQ/bgQ are null
+    // keep them null until it's time to created a SlotAcc, at which point we might inherit values
+    // from an ancestor facet context w/same key -- see comments in createSlotAcc
+    this.fgQ = fgQ;
+    this.bgQ = bgQ;
+
+    // TODO: defaults not supported yet -- see comments in createSlotAcc
+    if (null == fgQ || null == bgQ) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+                              NAME + " aggregate function requires both foreground & background " +
+                              "to be real (non-null) queries");
+    }
+  }
+
+  @Override
+  public String description() {
+    // TODO: need better output processing when we start supporting null fgQ/bgQ in constructor
+    return name +"(" + fgQ + "," + bgQ + ")";
+  }
+  
+  @Override
+  public boolean equals(Object o) {
+    if (! Objects.equals(this.getClass(), o.getClass())) {
+      return false;
+    }
+    RelatednessAgg that = (RelatednessAgg) o;
+    return Objects.equals(fgQ, that.fgQ) && Objects.equals(bgQ, that.bgQ);
+  }
+  
+  @Override
+  public int hashCode() {
+    return Objects.hash(getClass(), fgQ, bgQ);
+  }
+
+  @Override
+  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
+    throw new UnsupportedOperationException("NOT IMPLEMENTED " + name + " " + this);
+  }
+
+
+  public SlotAcc createSlotAcc(FacetContext fcontext, int numDocs, int numSlots) throws IOException {
+    // TODO: Ideally this is where we should check fgQ/bgQ for 'null' and apply defaults...
+    //
+    // we want to walk up the fcontext and inherit the queries from any ancestor SKGAgg
+    // with the same "key" that we have in our own context -- and as a last resort use
+    // "$q" for the foreground and "*:*" for the bgQ (if no ancestors)
+    // (Hmmm... or maybe we should use the "Domain" of our FacetRequest as the default bg?)
+    //
+    // How do we find our what key we have in the current context?
+    // loop over all the stats in the current context until we find one that's '==' to this???
+    
+    List<Query> fgFilters = new ArrayList<Query>(3);
+    fgFilters.add(fgQ);
+    for (FacetContext ctx = fcontext; ctx != null; ctx = ctx.parent) {
+      if (null != ctx.filter) {
+        fgFilters.add(ctx.filter);
+      } else {
+        // sanity check...
+        // the only way the filter on the current context should be null is...
+        assert (// 1) it's the actual top most context,
+                //    (ie: the func is directly used w/o being nested under a facet)
+                (null == ctx.parent && fcontext == ctx) ||
+                // 2) it's a child of the top most context
+                //    (ie: the context of a top level facet)
+                (null == ctx.parent.parent && null == ctx.parent.filter));
+        // either way, no reason to keep looping up the (0 or 1) remaining ancestors
+        // (which is why #1 can assert '&& fcontext == ctx')
+        break;
+      }
+    }
+    
+    return new SKGSlotAcc(fcontext, numSlots, fgFilters, bgQ);
+  }
+
+  @Override
+  public FacetMerger createFacetMerger(Object prototype) {
+    return new Merger();
+  }
+  
+  private static final class SKGSlotAcc extends SlotAcc {
+    private BucketData[] slotvalues;
+    private final DocSet fgSet;
+    private final DocSet bgSet;
+    private final long fgSize;
+    private final long bgSize;
+    private final List<Query> fgFilters;
+    private final Query bgQ;
+    public SKGSlotAcc(FacetContext fcontext, int numSlots,
+                      List<Query> fgFilters, Query bgQ) throws IOException {
+      super(fcontext);
+      this.fgFilters = fgFilters;
+      this.bgQ = bgQ;
+      this.fgSet = fcontext.searcher.getDocSet(fgFilters);
+      this.bgSet = fcontext.searcher.getDocSet(bgQ);
+      // cache the set sizes for frequent re-use on every slot
+      this.fgSize = fgSet.size();
+      this.bgSize = bgSet.size();
+      this.slotvalues = new BucketData[numSlots];
+      reset();
+    }
+
+    private void processSlot(int slot, IntFunction<SlotContext> slotContext) throws IOException {
+      
+      assert null != slotContext;
+      
+      Query slotQ = slotContext.apply(slot).getSlotQuery();
+      if (null == slotQ) {
+        // extremeley special edge case...
+        // the only way this should be possible is if our skg() function is used as a "top level" stat
+        // w/o being nested under any facet, in which case it should be a FacetQuery w/no parent...
+        assert fcontext.processor.freq instanceof FacetQuery : fcontext.processor.freq;
+        assert null == fcontext.parent;
+        assert null == fcontext.filter;
+      }
+      // ...and in which case we should just use the current base
+      final DocSet slotSet = null == slotQ ? fcontext.base : fcontext.searcher.getDocSet(slotQ);
+
+      final BucketData slotVal = new BucketData();
+      slotVal.incSizes(fgSize, bgSize);
+      slotVal.incCounts(fgSet.intersectionSize(slotSet),
+                        bgSet.intersectionSize(slotSet));
+      slotvalues[slot] = slotVal;
+    }
+    
+    @Override
+    public void collect(int perSegDocId, int slot, IntFunction<SlotContext> slotContext) throws IOException {
+      // NOTE: we don't actaully care about the individual docs being collected
+      // (the only reason we even bother implementing this method is because it's needed for sorting
+      // buckets by a function)
+      
+      // so we only worry about ensuring that every "slot" / bucket is processed the first time
+      // we're asked about it...
+      if (null == slotvalues[slot]) {
+        processSlot(slot, slotContext);
+      }
+    }
+
+    @Override
+    public int collect(DocSet docs, int slot, IntFunction<SlotContext> slotContext) throws IOException {
+      // NOTE: we don't actaully care about the doc set being collected for the bucket
+      // so we only worry about ensuring that every "slot" / bucket is processed exactly once
+      
+      // if we're doing bulk collection, we better not be getting asked to re-use slots
+      assert null == slotvalues[slot];
+      processSlot(slot, slotContext);
+
+      // we don't do any filtering, we collect the whole docset, so return that as out collected count
+      // (as a stat, we're actually required to return this by assertions in FacetFieldProcessor.processStats)
+      return docs.size();
+    }
+
+    public int compare(int slotA, int slotB) {
+      final BucketData a = slotvalues[slotA];
+      final BucketData b = slotvalues[slotB];
+      
+      // we initialize & reset() (unused) slotvalues elements to null
+      // but we should never be asked to compare a slot that hasn't been collected...
+      assert null != a;
+      assert null != b;
+      return a.compareTo(b);
+    }
+
+    @Override
+    public Object getValue(int slotNum) {
+      BucketData slotVal = slotvalues[slotNum];
+      if (null == slotVal) {
+        // since we haven't been told about any docs for this slot, use a slot w/no counts,
+        // just the known fg/bg sizes. (this is most likely a refinement request for a bucket we dont have)
+        slotVal = new BucketData();
+        slotVal.incSizes(fgSize, bgSize);
+      }
+
+      SimpleOrderedMap res = slotVal.externalize(fcontext.isShard());
+      return res;
+    }
+
+    @Override
+    public void reset() {
+      Arrays.fill(slotvalues, null);
+    }
+
+    @Override
+    public void resize(Resizer resizer) {
+      slotvalues = resizer.resize(slotvalues, null);
+    }
+
+    @Override
+    public void close() throws IOException {
+      slotvalues = null;
+    }
+  }
+  
+  /**
+   * Encapsulates all data needed for a single bucket/slot
+   * 
+   * @see SKGSlotAcc
+   * @see Merger
+   */
+  private static final class BucketData implements Comparable<BucketData> {
+    
+    private long fg_size = 0;
+    private long bg_size = 0;
+    private long fg_count = 0;
+    private long bg_count = 0;
+    private double relatedness = Double.NaN;
+    
+    public BucketData() {
+      /* No-Op */
+    }
+
+    /** 
+     * Increment both the foreground &amp; background <em>counts</em> for the current bucket, reseting any
+     * derived values that may be cached
+     */
+    public void incCounts(final long fgInc, final long bgInc) {
+        this.relatedness = Double.NaN;
+        fg_count += fgInc;
+        bg_count += bgInc;
+    }
+    /** 
+     * Increment both the foreground &amp; background <em>sizes</em> for the current bucket, reseting any
+     * derived values that may be cached
+     */
+    public void incSizes(final long fgInc, final long bgInc) {
+        this.relatedness = Double.NaN;
+        fg_size += fgInc;
+        bg_size += bgInc;
+    }
+    
+    @Override
+    public int hashCode() {
+      return Objects.hash(this.getClass(), fg_count, bg_count, fg_size, bg_size);
+    }
+    
+    @Override
+    public boolean equals(Object other) {
+      if (!Objects.equals(this.getClass(), other.getClass())) {
+        return false;
+      }
+      BucketData that = (BucketData)other;
+      // we will most certainly be compared to other buckets of the same Agg instance, so compare counts first
+      return Objects.equals(this.fg_count, that.fg_count)
+        && Objects.equals(this.bg_count, that.bg_count)
+        && Objects.equals(this.fg_size, that.fg_size)
+        && Objects.equals(this.bg_size, that.bg_size);
+    }
+
+    /**
+     * Computes (and caches) the derived relatedness score for this bucket
+     */
+    private double getRelatedness() {
+      if (Double.isNaN(this.relatedness)) {
+        this.relatedness = computeRelatedness(this.fg_count, this.fg_size,
+                                              this.bg_count, this.bg_size);
+        // TODO: add support for a "min_pop" option...
+        //
+        // if min_pop is configured, and either (fg|bg) popularity is lower then that value
+        // then "this.relatedness=-Infinity" so it sorts at the bottom
+        // this logic be ignored on isShard requests -- similar to how shards ignore 'mincount'
+      }
+      return this.relatedness;
+    }
+    
+    @Override
+    public int compareTo(BucketData that) {
+      // TODO: add support for a "sort_val" option...
+      //
+      // default should be "relatedness" but also support "foreground" and "background" ...
+      // either of those should sort by the corrisponding ratio
+      // To do this, we should probably precommpute the ratios in incCounts
+      
+      int r = Double.compare(this.getRelatedness(), that.getRelatedness());
+      if (0 == r) {
+        r = Long.compare(this.fg_count, that.fg_count);
+      }
+      if (0 == r) {
+        r = Long.compare(this.bg_count, that.bg_count);
+      }
+      return r;
+    }
+    
+    /**
+     * @see SlotAcc#getValue
+     * @see Merger#getMergedResult
+     */
+    public SimpleOrderedMap externalize(final boolean isShardRequest) {
+      SimpleOrderedMap result = new SimpleOrderedMap<Number>();
+      
+      if (isShardRequest) {
+        result.add(FG_COUNT, fg_count);
+        result.add(BG_COUNT, bg_count);
+        // NOTE: sizes will be the same for every slot...
+        // TODO: it would be nice to put them directly in the parent facet, instead of every bucket,
+        // in order to reduce the size of the response.
+        result.add(FG_SIZE, fg_size); 
+        result.add(BG_SIZE, bg_size);
+      } else {
+        // there's no need to bother computing these when returning results *to* a shard coordinator
+        // only useful to external clients 
+        result.add(RELATEDNESS, this.getRelatedness());
+        result.add(FG_POP, roundTo5Digits((double) fg_count / bg_size)); // yes, BACKGROUND size is intentional
+        result.add(BG_POP, roundTo5Digits((double) bg_count / bg_size));
+      }
+      
+      return result;
+    }
+  }
+
+  /**
+   * Merges in the per shard {@link BucketData} output into a unified {@link BucketData}
+   */
+  private static final class Merger extends FacetSortableMerger {
+    private final BucketData mergedData = new BucketData();
+    
+    @Override
+    public void merge(Object facetResult, Context mcontext) {
+      NamedList<Object> shardData = (NamedList<Object>)facetResult;
+      mergedData.incSizes((Long)shardData.remove(FG_SIZE), (Long)shardData.remove(BG_SIZE));
+      mergedData.incCounts((Long)shardData.remove(FG_COUNT), (Long)shardData.remove(BG_COUNT));
+    }
+
+    @Override
+    public int compareTo(FacetSortableMerger other, FacetRequest.SortDirection direction) {
+      // NOTE: regardless of the SortDirection hint, we want normal comparison of the BucketData
+      
+      assert other instanceof Merger;
+      Merger that = (Merger)other;
+      return mergedData.compareTo(that.mergedData);
+    }
+    
+    @Override
+    public Object getMergedResult() {
+      return mergedData.externalize(false);
+    }
+  }
+
+  
+  /**
+   * This is an aproximated Z-Score, as described in the "Scoring Semantic Relationships" 
+   * section of "<a href="https://arxiv.org/pdf/1609.00464.pdf">The Semantic Knowledge Graph: 
+   * A compact, auto-generated model for real-time traversal and ranking of any relationship 
+   * within a domain</a>"
+   *
+   * See Also:<ul>
+   * <li><a href="https://s.apache.org/Mfu2">java-user@lucene Message-ID: 449AEB60.4070300@alias-i.com</a></li>
+   * <li><a href="https://lingpipe-blog.com/2006/03/29/interesting-phrase-extraction-binomial-hypothesis-testing-vs-coding-loss/">Phrase Extraction: Binomial Hypothesis Testing vs. Coding Loss</a></li>
+   * </ul>
+   */
+  // NOTE: javadoc linter freaks out if we try doing those links as '@see <a href=...' tags
+  public static double computeRelatedness(final long fg_count, final long fg_size,
+                                          final long bg_count, final long bg_size) {
+    final double fg_size_d = (double) fg_size;
+    final double bg_size_d = (double) bg_size;
+    final double bg_prob = (bg_count / bg_size_d);
+    final double num = fg_count - fg_size_d * bg_prob;
+    double denom = Math.sqrt(fg_size_d * bg_prob * (1 - bg_prob));
+    denom = (denom == 0) ? 1e-10 : denom;
+    final double z = num / denom;
+    final double result = 0.2 * sigmoidHelper(z, -80, 50)
+      + 0.2 * sigmoidHelper(z, -30, 30)
+      + 0.2 * sigmoidHelper(z, 0, 30)
+      + 0.2 * sigmoidHelper(z, 30, 30)
+      + 0.2 * sigmoidHelper(z, 80, 50);
+    return roundTo5Digits(result);
+    
+  }
+  /**
+   * Helper function for rounding/truncating relatedness &amp; popularity values to 
+   * 5 decimal digits, since these values are all probabilistic more then 5 digits aren't really relevant
+   * and may give a missleading impression of added precision.
+   */
+  public static double roundTo5Digits(final double val) {
+    return Math.round(val * 1e5) / 1e5;
+  }
+  
+  /** A helper function for scaling values */
+  private static double sigmoidHelper(final double x, final double offset, final double scale) {
+    return (x+offset) / (scale + Math.abs(x+offset));
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java b/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java
index 9165799..9c12aff 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java
@@ -23,11 +23,13 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
+import java.util.function.IntFunction;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.Query;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.search.DocIterator;
@@ -65,9 +67,45 @@ public abstract class SlotAcc implements Closeable {
     }
   }
 
-  public abstract void collect(int doc, int slot) throws IOException;
+  /**
+   * @deprecated This method exists only for backcompatibility, developers of new {@link SlotAcc} 
+   * implementations should not implement this method, and should instead override 
+   * {@link #collect(int,int,IntFunction)}
+   */
+  @Deprecated
+  public void collect(int doc, int slot) throws IOException {
+    throw new UnsupportedOperationException
+      ("SlotAcc implementations must implement 'collect(int,int,IntFunction<SlotContext>)' or the (deprecated)"
+      + "'collect(int,int)'");
+  }
+  
+  /**
+   * All subclasses should override this method, for backcompatability the default implementaion
+   * delegates to the (deprecated) {@link #collect(int,int)}
+   *
+   * @param doc Single Segment docId (relative to the current {@link LeafReaderContext} to collect
+   * @param slot The slot number to collect this document in
+   * @param slotContext A callback that can be used for Accumulators that would like additional info 
+   *        about the current slot -- the {@link IntFunction} is only garunteed to be valid for 
+   *        the current slot, and the {@link SlotContext} returned is only valid for the duration 
+   *        of the <code>collect()</code> call.
+   */
+  @Deprecated
+  public void collect(int doc, int slot, IntFunction<SlotContext> slotContext) throws IOException {
+    collect(doc,slot);
+  }
 
-  public int collect(DocSet docs, int slot) throws IOException {
+  /**
+   * Bulk collection of all documents in a slot.  The default implementation calls {@link #collect(int,int,IntFunction)}
+   *
+   * @param docs (global) Documents to collect
+   * @param slot The slot number to collect these documents in
+   * @param slotContext A callback that can be used for Accumulators that would like additional info 
+   *        about the current slot -- the {@link IntFunction} is only garunteed to be valid for 
+   *        the current slot, and the {@link SlotContext} returned is only valid for the duration 
+   *        of the <code>collect()</code> call.
+   */
+  public int collect(DocSet docs, int slot, IntFunction<SlotContext> slotContext) throws IOException {
     int count = 0;
     SolrIndexSearcher searcher = fcontext.searcher;
 
@@ -94,7 +132,7 @@ public abstract class SlotAcc implements Closeable {
         setNextReader(ctx);
       }
       count++;
-      collect(doc - segBase, slot); // per-seg collectors
+      collect(doc - segBase, slot, slotContext); // per-seg collectors
     }
     return count;
   }
@@ -212,6 +250,19 @@ public abstract class SlotAcc implements Closeable {
 
   } // end class Resizer
 
+  /**
+   * Incapsulates information about the current slot, for Accumulators that may want 
+   * additional info during collection.
+   */
+  public static final class SlotContext {
+    private final Query slotQuery;
+    public SlotContext(Query slotQuery) {
+      this.slotQuery = slotQuery;
+    }
+    public Query getSlotQuery() {
+      return slotQuery;
+    }
+  }
 }
 
 // TODO: we should really have a decoupled value provider...
@@ -349,7 +400,7 @@ class SumSlotAcc extends DoubleFuncSlotAcc {
     super(values, fcontext, numSlots);
   }
 
-  public void collect(int doc, int slotNum) throws IOException {
+  public void collect(int doc, int slotNum, IntFunction<SlotContext> slotContext) throws IOException {
     double val = values.doubleVal(doc); // todo: worth trying to share this value across multiple stats that need it?
     result[slotNum] += val;
   }
@@ -361,7 +412,7 @@ class SumsqSlotAcc extends DoubleFuncSlotAcc {
   }
 
   @Override
-  public void collect(int doc, int slotNum) throws IOException {
+  public void collect(int doc, int slotNum, IntFunction<SlotContext> slotContext) throws IOException {
     double val = values.doubleVal(doc);
     val = val * val;
     result[slotNum] += val;
@@ -386,7 +437,7 @@ class AvgSlotAcc extends DoubleFuncSlotAcc {
   }
 
   @Override
-  public void collect(int doc, int slotNum) throws IOException {
+  public void collect(int doc, int slotNum, IntFunction<SlotContext> slotContext) throws IOException {
     double val = values.doubleVal(doc);
     if (val != 0 || values.exists(doc)) {
       result[slotNum] += val;
@@ -479,7 +530,7 @@ class VarianceSlotAcc extends DoubleFuncSlotAcc {
   }
 
   @Override
-  public void collect(int doc, int slot) throws IOException {
+  public void collect(int doc, int slot, IntFunction<SlotContext> slotContext) throws IOException {
     double val = values.doubleVal(doc);
     if (values.exists(doc)) {
       counts[slot]++;
@@ -541,7 +592,7 @@ class StddevSlotAcc extends DoubleFuncSlotAcc {
   }
 
   @Override
-  public void collect(int doc, int slot) throws IOException {
+  public void collect(int doc, int slot, IntFunction<SlotContext> slotContext) throws IOException {
     double val = values.doubleVal(doc);
     if (values.exists(doc)) {
       counts[slot]++;
@@ -570,8 +621,9 @@ class CountSlotArrAcc extends CountSlotAcc {
   }
 
   @Override
-  public void collect(int doc, int slotNum) { // TODO: count arrays can use fewer bytes based on the number of docs in
-                                              // the base set (that's the upper bound for single valued) - look at ttf?
+  public void collect(int doc, int slotNum, IntFunction<SlotContext> slotContext) {
+    // TODO: count arrays can use fewer bytes based on the number of docs in
+    // the base set (that's the upper bound for single valued) - look at ttf?
     result[slotNum]++;
   }
 
@@ -615,7 +667,7 @@ class SortSlotAcc extends SlotAcc {
   }
 
   @Override
-  public void collect(int doc, int slot) throws IOException {
+  public void collect(int doc, int slot, IntFunction<SlotContext> slotContext) throws IOException {
     // no-op
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/UnInvertedField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/UnInvertedField.java b/solr/core/src/java/org/apache/solr/search/facet/UnInvertedField.java
index 69b341a..3349bb2 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/UnInvertedField.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/UnInvertedField.java
@@ -44,6 +44,7 @@ import org.apache.solr.search.DocIterator;
 import org.apache.solr.search.DocSet;
 import org.apache.solr.search.SolrCache;
 import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.search.facet.SlotAcc.SlotContext;
 import org.apache.solr.uninverting.DocTermOrds;
 import org.apache.solr.util.TestInjection;
 import org.slf4j.Logger;
@@ -431,7 +432,8 @@ public class UnInvertedField extends DocTermOrds {
       if (tt.termNum >= startTermIndex && tt.termNum < endTermIndex) {
         // handle the biggest terms
         DocSet intersection = searcher.getDocSet(tt.termQuery, docs);
-        int collected = processor.collectFirstPhase(intersection, tt.termNum - startTermIndex);
+        int collected = processor.collectFirstPhase(intersection, tt.termNum - startTermIndex,
+                                                    slotNum -> { return new SlotContext(tt.termQuery); });
         countAcc.incrementCount(tt.termNum - startTermIndex, collected);
         if (collected > 0) {
           uniqueTerms++;
@@ -493,7 +495,7 @@ public class UnInvertedField extends DocTermOrds {
             if (arrIdx < 0) continue;
             if (arrIdx >= nTerms) break;
             countAcc.incrementCount(arrIdx, 1);
-            processor.collectFirstPhase(segDoc, arrIdx);
+            processor.collectFirstPhase(segDoc, arrIdx, processor.slotContext);
           }
         } else {
           int tnum = 0;
@@ -507,7 +509,7 @@ public class UnInvertedField extends DocTermOrds {
               if (arrIdx >= 0) {
                 if (arrIdx >= nTerms) break;
                 countAcc.incrementCount(arrIdx, 1);
-                processor.collectFirstPhase(segDoc, arrIdx);
+                processor.collectFirstPhase(segDoc, arrIdx, processor.slotContext);
               }
               delta = 0;
             }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java b/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java
index d858b5b..5d9cf90 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java
@@ -21,6 +21,7 @@ import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.function.IntFunction;
 
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
@@ -145,7 +146,7 @@ public class UniqueAgg extends StrAggValueSource {
     }
 
     @Override
-    public void collect(int doc, int slot) throws IOException {
+    public void collect(int doc, int slot, IntFunction<SlotContext> slotContext) throws IOException {
       int valuesDocID = docIdSetIterator().docID();
       if (valuesDocID < doc) {
         valuesDocID = docIdSetIterator().advance(doc);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/UniqueMultiDvSlotAcc.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/UniqueMultiDvSlotAcc.java b/solr/core/src/java/org/apache/solr/search/facet/UniqueMultiDvSlotAcc.java
index af419a4..839fc52 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/UniqueMultiDvSlotAcc.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/UniqueMultiDvSlotAcc.java
@@ -18,6 +18,7 @@
 package org.apache.solr.search.facet;
 
 import java.io.IOException;
+import java.util.function.IntFunction;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.MultiDocValues;
@@ -70,7 +71,7 @@ class UniqueMultiDvSlotAcc extends UniqueSlotAcc {
   }
 
   @Override
-  public void collect(int doc, int slotNum) throws IOException {
+  public void collect(int doc, int slotNum, IntFunction<SlotContext> slotContext) throws IOException {
     if (subDv.advanceExact(doc)) {
 
       int segOrd = (int) subDv.nextOrd();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/UniqueMultivaluedSlotAcc.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/UniqueMultivaluedSlotAcc.java b/solr/core/src/java/org/apache/solr/search/facet/UniqueMultivaluedSlotAcc.java
index 10adcf0..508da38 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/UniqueMultivaluedSlotAcc.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/UniqueMultivaluedSlotAcc.java
@@ -18,6 +18,7 @@
 package org.apache.solr.search.facet;
 
 import java.io.IOException;
+import java.util.function.IntFunction;
 
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
@@ -50,7 +51,7 @@ class UniqueMultivaluedSlotAcc extends UniqueSlotAcc implements UnInvertedField.
   }
 
   @Override
-  public void collect(int doc, int slotNum) throws IOException {
+  public void collect(int doc, int slotNum, IntFunction<SlotContext> slotContext) throws IOException {
     bits = arr[slotNum];
     if (bits == null) {
       bits = new FixedBitSet(nTerms);
@@ -67,4 +68,4 @@ class UniqueMultivaluedSlotAcc extends UniqueSlotAcc implements UnInvertedField.
       docToTerm = null;
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0d6a0e6/solr/core/src/java/org/apache/solr/search/facet/UniqueSinglevaluedSlotAcc.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/UniqueSinglevaluedSlotAcc.java b/solr/core/src/java/org/apache/solr/search/facet/UniqueSinglevaluedSlotAcc.java
index 9a1b51e..ed51c5b 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/UniqueSinglevaluedSlotAcc.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/UniqueSinglevaluedSlotAcc.java
@@ -18,6 +18,7 @@
 package org.apache.solr.search.facet;
 
 import java.io.IOException;
+import java.util.function.IntFunction;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.MultiDocValues;
@@ -73,7 +74,7 @@ class UniqueSinglevaluedSlotAcc extends UniqueSlotAcc {
   }
 
   @Override
-  public void collect(int doc, int slotNum) throws IOException {
+  public void collect(int doc, int slotNum, IntFunction<SlotContext> slotContext) throws IOException {
     if (doc > subDv.docID()) {
       subDv.advance(doc);
     }