You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2016/09/07 09:57:31 UTC

[43/50] [abbrv] lucene-solr:apiv2: SOLR-5725: facet.exists=true caps counts by 1 to make facet.method=enum faster.

SOLR-5725:  facet.exists=true caps counts by 1 to make facet.method=enum
faster.


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

Branch: refs/heads/apiv2
Commit: ff69d14868555c43708823df23c90abd58a14d86
Parents: abd4cfb
Author: Mikhail Khludnev <mk...@apache.org>
Authored: Sat Aug 27 22:52:39 2016 +0300
Committer: Mikhail Khludnev <mk...@apache.org>
Committed: Sat Sep 3 07:24:48 2016 +0300

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../solr/handler/component/FacetComponent.java  |  31 +-
 .../org/apache/solr/request/SimpleFacets.java   |  71 ++++-
 .../apache/solr/search/SolrIndexSearcher.java   |   5 +
 .../org/apache/solr/TestRandomFaceting.java     | 261 ++++++++++++++---
 .../DistributedFacetExistsSmallTest.java        | 227 +++++++++++++++
 .../apache/solr/request/SimpleFacetsTest.java   | 286 ++++++++++++++++++-
 .../apache/solr/common/params/FacetParams.java  |   8 +
 8 files changed, 842 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ff69d148/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 3b220d2..0d507e3 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -77,6 +77,8 @@ prefix, then you will now get an error as these options are incompatible with nu
 
 New Features
 ----------------------
+* SOLR-5725: facet.method=enum can bypass exact counts calculation with facet.exists=true, it just returns 1 for 
+  terms which exists in result docset. (Alexey Kozhemiakin, Sebastian Koziel, Radoslaw Zielinski via Mikhail Khludnev) 
 
 Bug Fixes
 ----------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ff69d148/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java b/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java
index 26b2e59..90608c0 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java
@@ -1265,7 +1265,14 @@ public class FacetComponent extends SearchComponent {
       if (facetFs != null) {
         
         for (String field : facetFs) {
-          DistribFieldFacet ff = new DistribFieldFacet(rb, field);
+          final DistribFieldFacet ff;
+          
+          if (params.getFieldBool(field, FacetParams.FACET_EXISTS, false)) {
+            // cap facet count by 1 with this method
+            ff = new DistribFacetExistsField(rb, field);
+          } else {
+            ff = new DistribFieldFacet(rb, field);
+          }
           facets.put(ff.getKey(), ff);
         }
       }
@@ -1469,7 +1476,7 @@ public class FacetComponent extends SearchComponent {
             sfc.termNum = termNum++;
             counts.put(name, sfc);
           }
-          sfc.count += count;
+          incCount(sfc, count);
           terms.set(sfc.termNum);
           last = count;
         }
@@ -1485,6 +1492,10 @@ public class FacetComponent extends SearchComponent {
       missingMax[shardNum] = last;
       counted[shardNum] = terms;
     }
+
+    protected void incCount(ShardFacetCount sfc, long count) {
+      sfc.count += count;
+    }
     
     public ShardFacetCount[] getLexSorted() {
       ShardFacetCount[] arr 
@@ -1530,7 +1541,7 @@ public class FacetComponent extends SearchComponent {
       }
     }
   }
-  
+
   /**
    * <b>This API is experimental and subject to change</b>
    */
@@ -1547,4 +1558,18 @@ public class FacetComponent extends SearchComponent {
     }
   }
 
+  
+  private static final class DistribFacetExistsField extends DistribFieldFacet {
+    private DistribFacetExistsField(ResponseBuilder rb, String facetStr) {
+      super(rb, facetStr);
+      SimpleFacets.checkMincountOnExists(field, minCount); 
+    }
+
+    @Override
+    protected void incCount(ShardFacetCount sfc, long count) {
+      if (count>0) {
+        sfc.count = 1;
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ff69d148/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/SimpleFacets.java b/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
index c2f68f9..52c2129 100644
--- a/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
+++ b/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
@@ -406,7 +406,8 @@ public class SimpleFacets {
     String prefix = params.getFieldParam(field, FacetParams.FACET_PREFIX);
     String contains = params.getFieldParam(field, FacetParams.FACET_CONTAINS);
     boolean ignoreCase = params.getFieldBool(field, FacetParams.FACET_CONTAINS_IGNORE_CASE, false);
-
+    boolean exists = params.getFieldBool(field, FacetParams.FACET_EXISTS, false);
+    
     NamedList<Integer> counts;
     SchemaField sf = searcher.getSchema().getField(field);
     FieldType ft = sf.getType();
@@ -422,13 +423,15 @@ public class SimpleFacets {
       requestedMethod = FacetMethod.FC;
     } else if(FacetParams.FACET_METHOD_uif.equals(methodStr)) {
       requestedMethod = FacetMethod.UIF;
-    }else{
+    } else {
       requestedMethod=null;
     }
 
     final boolean multiToken = sf.multiValued() || ft.multiValuedFieldCache();
 
-    FacetMethod appliedFacetMethod = selectFacetMethod(sf, requestedMethod, mincount);
+    FacetMethod appliedFacetMethod = selectFacetMethod(field,
+                                sf, requestedMethod, mincount,
+                                exists);
 
     RTimer timer = null;
     if (fdebug != null) {
@@ -446,7 +449,8 @@ public class SimpleFacets {
       switch (appliedFacetMethod) {
         case ENUM:
           assert TrieField.getMainValuePrefix(ft) == null;
-          counts = getFacetTermEnumCounts(searcher, docs, field, offset, limit, mincount,missing,sort,prefix, contains, ignoreCase, params);
+          counts = getFacetTermEnumCounts(searcher, docs, field, offset, limit, mincount,missing,sort,prefix, contains, ignoreCase, 
+                                          exists);
           break;
         case FCS:
           assert !multiToken;
@@ -538,6 +542,29 @@ public class SimpleFacets {
     return counts;
   }
 
+   /**
+    * @param existsRequested facet.exists=true is passed for the given field
+    * */
+  static FacetMethod selectFacetMethod(String fieldName, 
+                                       SchemaField field, FacetMethod method, Integer mincount,
+                                       boolean existsRequested) {
+    if (existsRequested) {
+      checkMincountOnExists(fieldName, mincount);
+      if (method == null) {
+        method = FacetMethod.ENUM;
+      }
+    }
+    final FacetMethod facetMethod = selectFacetMethod(field, method, mincount);
+    
+    if (existsRequested && facetMethod!=FacetMethod.ENUM) {
+      throw new SolrException (ErrorCode.BAD_REQUEST, 
+          FacetParams.FACET_EXISTS + "=true is requested, but "+
+          FacetParams.FACET_METHOD+"="+FacetParams.FACET_METHOD_enum+ " can't be used with "+fieldName
+      );
+    }
+    return facetMethod;
+  }
+    
   /**
    * This method will force the appropriate facet method even if the user provided a different one as a request parameter
    *
@@ -811,7 +838,8 @@ public class SimpleFacets {
    * @see FacetParams#FACET_ZEROS
    * @see FacetParams#FACET_MISSING
    */
-  public NamedList<Integer> getFacetTermEnumCounts(SolrIndexSearcher searcher, DocSet docs, String field, int offset, int limit, int mincount, boolean missing, String sort, String prefix, String contains, boolean ignoreCase, SolrParams params)
+  public NamedList<Integer> getFacetTermEnumCounts(SolrIndexSearcher searcher, DocSet docs, String field, int offset, int limit, int mincount, boolean missing, 
+                                      String sort, String prefix, String contains, boolean ignoreCase, boolean intersectsCheck)
     throws IOException {
 
     /* :TODO: potential optimization...
@@ -901,7 +929,11 @@ public class SimpleFacets {
                 deState.postingsEnum = postingsEnum;
               }
 
-              c = searcher.numDocs(docs, deState);
+              if (intersectsCheck) {
+                c = searcher.intersects(docs, deState) ? 1 : 0;
+              } else {
+                c = searcher.numDocs(docs, deState);
+              }
 
               postingsEnum = deState.postingsEnum;
             } else {
@@ -916,19 +948,33 @@ public class SimpleFacets {
               if (postingsEnum instanceof MultiPostingsEnum) {
                 MultiPostingsEnum.EnumWithSlice[] subs = ((MultiPostingsEnum) postingsEnum).getSubs();
                 int numSubs = ((MultiPostingsEnum) postingsEnum).getNumSubs();
+                
+                SEGMENTS_LOOP:
                 for (int subindex = 0; subindex < numSubs; subindex++) {
                   MultiPostingsEnum.EnumWithSlice sub = subs[subindex];
                   if (sub.postingsEnum == null) continue;
                   int base = sub.slice.start;
                   int docid;
                   while ((docid = sub.postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-                    if (fastForRandomSet.exists(docid + base)) c++;
+                    if (fastForRandomSet.exists(docid + base)) {
+                      c++;
+                      if (intersectsCheck) {
+                        assert c==1;
+                        break SEGMENTS_LOOP;
+                      }
+                    }
                   }
                 }
               } else {
                 int docid;
                 while ((docid = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-                  if (fastForRandomSet.exists(docid)) c++;
+                  if (fastForRandomSet.exists(docid)) {
+                    c++;
+                    if (intersectsCheck) {
+                      assert c==1;
+                      break;
+                    }
+                  }
                 }
               }
 
@@ -969,6 +1015,15 @@ public class SimpleFacets {
     return res;
   }
 
+  public static void checkMincountOnExists(String fieldName, int mincount) {
+    if (mincount > 1) {
+        throw new SolrException (ErrorCode.BAD_REQUEST,
+            FacetParams.FACET_MINCOUNT + "="+mincount+" exceed 1 that's not supported with " + 
+                FacetParams.FACET_EXISTS + "=true for " + fieldName
+        );
+      }
+  }
+
   /**
    * A simple key=&gt;val pair whose natural order is such that 
    * <b>higher</b> vals come before lower vals.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ff69d148/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index 7f15574..4c18809 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -2285,6 +2285,11 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     return all.andNotSize(positiveA.union(positiveB));
   }
 
+  /** @lucene.internal */
+  public boolean intersects(DocSet a, DocsEnumState deState) throws IOException {
+    return a.intersects(getDocSet(deState));
+  }
+
   /**
    * Takes a list of document IDs, and returns an array of Documents containing all of the stored fields.
    */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ff69d148/solr/core/src/test/org/apache/solr/TestRandomFaceting.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/TestRandomFaceting.java b/solr/core/src/test/org/apache/solr/TestRandomFaceting.java
index daafca1..2ffefdc 100644
--- a/solr/core/src/test/org/apache/solr/TestRandomFaceting.java
+++ b/solr/core/src/test/org/apache/solr/TestRandomFaceting.java
@@ -16,22 +16,39 @@
  */
 package org.apache.solr;
 
-import org.apache.lucene.util.TestUtil;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.regex.Pattern;
+
 import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.schema.SchemaField;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.noggit.JSONUtil;
+import org.noggit.ObjectBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.lang.invoke.MethodHandles;
-import java.util.*;
-
 @Slow
 public class TestRandomFaceting extends SolrTestCaseJ4 {
 
+  private static final Pattern trieFields = Pattern.compile(".*_t.");
+
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   public static final String FOO_STRING_FIELD = "foo_s1";
@@ -80,6 +97,21 @@ public class TestRandomFaceting extends SolrTestCaseJ4 {
     types.add(new FldType("missing_ss",new IRange(0,0), new SVal('a','b',1,1)));
 
     // TODO: doubles, multi-floats, ints with precisionStep>0, booleans
+    types.add(new FldType("small_tf",ZERO_ONE, new FVal(-4,5)));
+    assert trieFields.matcher("small_tf").matches();
+    assert !trieFields.matcher("small_f").matches();
+    
+    types.add(new FldType("foo_ti",ZERO_ONE, new IRange(-2,indexSize)));
+    assert trieFields.matcher("foo_ti").matches();
+    assert !trieFields.matcher("foo_i").matches();
+    
+    types.add(new FldType("bool_b",ZERO_ONE, new Vals(){
+      @Override
+      public Comparable get() {
+        return random().nextBoolean();
+      }
+      
+    }));
   }
 
   void addMoreDocs(int ndocs) throws Exception {
@@ -144,8 +176,8 @@ public class TestRandomFaceting extends SolrTestCaseJ4 {
   }
 
 
-  List<String> multiValuedMethods = Arrays.asList(new String[]{"enum","fc"});
-  List<String> singleValuedMethods = Arrays.asList(new String[]{"enum","fc","fcs"});
+  List<String> multiValuedMethods = Arrays.asList(new String[]{"enum","fc", null});
+  List<String> singleValuedMethods = Arrays.asList(new String[]{"enum","fc","fcs", null});
 
 
   void doFacetTests(FldType ftype) throws Exception {
@@ -154,10 +186,9 @@ public class TestRandomFaceting extends SolrTestCaseJ4 {
       Random rand = random();
       boolean validate = validateResponses;
       ModifiableSolrParams params = params("facet","true", "wt","json", "indent","true", "omitHeader","true");
-      params.add("q","*:*", "rows","0");  // TODO: select subsets
+      params.add("q","*:*");  // TODO: select subsets
       params.add("rows","0");
 
-
       SchemaField sf = req.getSchema().getField(ftype.fname);
       boolean multiValued = sf.getType().multiValuedFieldCache();
 
@@ -198,6 +229,10 @@ public class TestRandomFaceting extends SolrTestCaseJ4 {
         params.add("facet.missing", "true");
       }
 
+      if (rand.nextBoolean()) {
+        params.add("facet.enum.cache.minDf",""+ rand.nextInt(indexSize));
+      }
+      
       // TODO: randomly add other facet params
       String key = ftype.fname;
       String facet_field = ftype.fname;
@@ -210,45 +245,207 @@ public class TestRandomFaceting extends SolrTestCaseJ4 {
       List<String> methods = multiValued ? multiValuedMethods : singleValuedMethods;
       List<String> responses = new ArrayList<>(methods.size());
       for (String method : methods) {
-        // params.add("facet.field", "{!key="+method+"}" + ftype.fname);
-        // TODO: allow method to be passed on local params?
-
-        params.set("facet.method", method);
-
-        // if (random().nextBoolean()) params.set("facet.mincount", "1");  // uncomment to test that validation fails
-
-        String strResponse = h.query(req(params));
-        // Object realResponse = ObjectBuilder.fromJSON(strResponse);
-        // System.out.println(strResponse);
-
-        responses.add(strResponse);
+        for (boolean exists : new boolean [] {false, true}) {
+          // params.add("facet.field", "{!key="+method+"}" + ftype.fname);
+          // TODO: allow method to be passed on local params?
+          if (method!=null) {
+            params.set("facet.method", method);
+          } else {
+            params.remove("facet.method");
+          }
+          
+          params.set("facet.exists", ""+exists);
+          if (!exists && rand.nextBoolean()) {
+            params.remove("facet.exists");
+          }
+          
+          // if (random().nextBoolean()) params.set("facet.mincount", "1");  // uncomment to test that validation fails
+          if (params.getInt("facet.limit", 100)!=0) { // it bypasses all processing, and we can go to empty validation
+            if (exists && params.getInt("facet.mincount", 0)>1) {
+              assertQEx("no mincount on facet.exists",
+                  rand.nextBoolean() ? "facet.exists":"facet.mincount",
+                  req(params), ErrorCode.BAD_REQUEST);
+              continue;
+            }
+            // facet.exists can't be combined with non-enum nor with enum requested for tries, because it will be flipped to FC/FCS 
+            final boolean notEnum = method != null && !method.equals("enum");
+            final boolean trieField = trieFields.matcher(ftype.fname).matches();
+            if ((notEnum || trieField) && exists) {
+              assertQEx("facet.exists only when enum or ommitted", 
+                  "facet.exists", req(params), ErrorCode.BAD_REQUEST);
+              continue;
+            }
+          }
+          String strResponse = h.query(req(params));
+          responses.add(strResponse);
+          
+          if (responses.size()>1) {
+            validateResponse(responses.get(0), strResponse, params, method, methods);
+          }
+        }
+        
       }
-
+      
       /**
       String strResponse = h.query(req(params));
       Object realResponse = ObjectBuilder.fromJSON(strResponse);
       **/
+    } finally {
+      req.close();
+    }
+  }
+  private void validateResponse(String expected, String actual, ModifiableSolrParams params, String method,
+        List<String> methods) throws Exception {
+    if (params.getBool("facet.exists", false)) {
+      if (isSortByCount(params)) { // it's challenged with facet.sort=count 
+        expected = getExpectationForSortByCount(params, methods);// that requires to recalculate expactation
+      } else { // facet.sort=index
+        expected = capFacetCountsTo1(expected);
+      }
+    }
+    
+    String err = JSONTestUtil.match("/", actual, expected, 0.0);
+    if (err != null) {
+      log.error("ERROR: mismatch facet response: " + err +
+          "\n expected =" + expected +
+          "\n response = " + actual +
+          "\n request = " + params
+      );
+      fail(err);
+    }
+  }
 
-      if (validate) {
-        for (int i=1; i<methods.size(); i++) {
-          String err = JSONTestUtil.match("/", responses.get(i), responses.get(0), 0.0);
-          if (err != null) {
-            log.error("ERROR: mismatch facet response: " + err +
-                "\n expected =" + responses.get(0) +
-                "\n response = " + responses.get(i) +
-                "\n request = " + params
-            );
-            fail(err);
+  /** if facet.exists=true with facet.sort=counts,
+   * it should return all values with 1 hits ordered by label index
+   * then all vals with 0 , and then missing count with null label,
+   * in the implementation below they are called three stratas 
+   * */
+  private String getExpectationForSortByCount( ModifiableSolrParams params, List<String> methods) throws Exception {
+    String indexSortedResponse = getIndexSortedAllFacetValues(params, methods);
+    
+    return transformFacetFields(indexSortedResponse, e -> {
+      List<Object> facetSortedByIndex = (List<Object>) e.getValue();
+      Map<Integer,List<Object>> stratas = new HashMap<Integer,List<Object>>(){
+        @Override // poor man multimap, I won't do that anymore, I swear.
+        public List<Object> get(Object key) {
+          if (!containsKey(key)) {
+            put((Integer) key, new ArrayList<>());
           }
+          return super.get(key);
         }
+      };
+      
+      for (Iterator iterator = facetSortedByIndex.iterator(); iterator.hasNext();) {
+        Object label = (Object) iterator.next();
+        Long count = (Long) iterator.next();
+        final Integer strata;
+        if (label==null) { // missing (here "stratas" seems like overengineering )
+          strata = null;
+        }else {
+          if (count>0) {
+            count = 1L; // capping here 
+            strata = 1; // non-zero count become zero
+          } else {
+            strata = 0; // zero-count
+          }
+        }
+        final List<Object> facet = stratas.get(strata);
+        facet.add(label);
+        facet.add(count);
       }
+      List stratified =new ArrayList<>();
+      for(Integer s : new Integer[]{1, 0}) { // non-zero capped to one goes first, zeroes go then
+        stratified.addAll(stratas.get(s));
+      }// cropping them now
+      int offset=params.getInt("facet.offset", 0) * 2;
+      int end = offset + params.getInt("facet.limit", 100) * 2 ;
+      int fromIndex = offset > stratified.size() ?  stratified.size() : offset;
+      stratified = stratified.subList(fromIndex, 
+               end > stratified.size() ?  stratified.size() : end);
+      
+      if (params.getInt("facet.limit", 100)>0) { /// limit=0 omits even miss count
+        stratified.addAll(stratas.get(null));
+      }
+      facetSortedByIndex.clear();
+      facetSortedByIndex.addAll(stratified);
+    });
+  }
 
-
+  private String getIndexSortedAllFacetValues(ModifiableSolrParams in, List<String> methods) throws Exception {
+    ModifiableSolrParams params = new ModifiableSolrParams(in);
+    params.set("facet.sort", "index");
+    String goodOldMethod = methods.get(random().nextInt( methods.size()));
+    params.set("facet.method", goodOldMethod);
+    params.set("facet.exists", "false");
+    if (random().nextBoolean()) {
+      params.remove("facet.exists");
+    }
+    params.set("facet.limit",-1);
+    params.set("facet.offset",0);
+    final String query;
+    SolrQueryRequest req = null;
+    try {
+      req = req(params);
+      query = h.query(req);
     } finally {
       req.close();
     }
+    return query;
   }
 
+  private boolean isSortByCount(ModifiableSolrParams in) {
+    boolean sortIsCount;
+    String sortParam = in.get("facet.sort");
+    sortIsCount = "count".equals(sortParam) || (sortParam==null && in.getInt("facet.limit",100)>0);
+    return sortIsCount;
+  }
+
+  /*
+   * {
+  "response":{"numFound":6,"start":0,"docs":[]
+  },
+  "facet_counts":{
+    "facet_queries":{},
+    "facet_fields":{
+      "foo_i":[
+        "6",2,
+        "2",1,
+        "3",1]},
+    "facet_ranges":{},
+    "facet_intervals":{},
+    "facet_heatmaps":{}}} 
+   * */
+  @SuppressWarnings({"rawtypes", "unchecked"})
+  private String capFacetCountsTo1(String expected) throws IOException {
+    return transformFacetFields(expected, e -> {
+      List<Object> facetValues = (List<Object>) e.getValue();
+      for (ListIterator iterator = facetValues.listIterator(); iterator.hasNext();) {
+        Object value = iterator.next(); 
+        Long count = (Long) iterator.next();
+        if (value!=null && count > 1) {
+          iterator.set(1);
+        }
+        
+      }
+    });
+  }
+  
+  private String transformFacetFields(String expected, Consumer<Map.Entry<Object,Object>> consumer) throws IOException {
+    Object json = ObjectBuilder.fromJSON(expected);
+    Map facet_fields = getFacetFieldMap(json);
+    Set entries = facet_fields.entrySet();
+    for (Object facetTuples : entries) { //despite there should be only one field
+      Entry entry = (Entry)facetTuples;
+      consumer.accept(entry);
+    }
+    return JSONUtil.toJSON(json);
+  }
+
+  private Map getFacetFieldMap(Object json) {
+    Object facet_counts = ((Map)json).get("facet_counts");
+    Map facet_fields = (Map) ((Map)facet_counts).get("facet_fields");
+    return facet_fields;
+  }
 }
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ff69d148/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetExistsSmallTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetExistsSmallTest.java b/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetExistsSmallTest.java
new file mode 100644
index 0000000..4a827be
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetExistsSmallTest.java
@@ -0,0 +1,227 @@
+/*
+ * 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.handler.component;
+
+import static org.hamcrest.CoreMatchers.is;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.solr.BaseDistributedSearchTestCase;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.response.FacetField;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.junit.Before;
+
+public class DistributedFacetExistsSmallTest extends BaseDistributedSearchTestCase {
+
+  public static final String FLD = "t_s";
+  private int maxId;
+  
+  public DistributedFacetExistsSmallTest() {
+  }
+
+  @Before
+  public void prepareIndex() throws Exception {
+    del("*:*");
+
+    final Random rnd = random();
+    index(id, maxId=rnd.nextInt(5), FLD, "AAA");
+    index(id, maxId+=1+rnd.nextInt(5), FLD, "B");
+    index(id, maxId+=1+rnd.nextInt(5), FLD, "BB");
+    index(id, maxId+=1+rnd.nextInt(5), FLD, "BB");
+    index(id, maxId+=1+rnd.nextInt(5), FLD, "BBB");
+    index(id, maxId+=1+rnd.nextInt(5), FLD, "BBB");
+    index(id, maxId+=1+rnd.nextInt(5), FLD, "BBB");
+    index(id, maxId+=1+rnd.nextInt(5), FLD, "CC");
+    index(id, maxId+=1+rnd.nextInt(5), FLD, "CC");
+    index(id, maxId+=1+rnd.nextInt(5), FLD, "CCC");
+    index(id, maxId+=1+rnd.nextInt(5), FLD, "CCC");
+    index(id, maxId+=1+rnd.nextInt(5), FLD, "CCC");
+
+    final SolrClient shard0 = clients.get(0);
+    // expectidly fails test
+    //shard0.add(sdoc("id", 13, FLD, "DDD"));
+    commit();
+
+    handle.clear();
+    handle.put("QTime", SKIPVAL);
+    handle.put("timestamp", SKIPVAL);
+    handle.put("maxScore", SKIPVAL);
+    handle.put("_version_", SKIPVAL);
+  }
+
+  @ShardsFixed(num=4)
+  public void test() throws Exception{
+    checkBasicRequest();
+    checkWithMinCountEqOne();
+    checkWithSortCount();
+    checkWithMethodSetPerField();
+    
+    {
+      // empty enum for checking npe
+      final ModifiableSolrParams params = buildParams();
+      params.remove("facet.exists");
+      QueryResponse rsp = query(params);
+    }
+    
+    checkRandomParams();
+    
+    checkInvalidMincount();
+  }
+
+  private void checkRandomParams() throws Exception {
+    final ModifiableSolrParams params = buildParams();
+    Random rand = random();
+
+    if (rand.nextBoolean()) {
+      int from;
+      params.set("q", "["+(from = rand.nextInt(maxId/2))+
+                  " TO "+((from-1)+(rand.nextInt(maxId)))+"]");
+    }
+    
+    int offset = 0;
+    int indexSize = 6;
+    if (rand .nextInt(100) < 20) {
+      if (rand.nextBoolean()) {
+        offset = rand.nextInt(100) < 10 ? rand.nextInt(indexSize *2) : rand.nextInt(indexSize/3+1);
+      }
+      params.add("facet.offset", Integer.toString(offset));
+    }
+
+    int limit = 100;
+    if (rand.nextInt(100) < 20) {
+      if (rand.nextBoolean()) {
+        limit = rand.nextInt(100) < 10 ? rand.nextInt(indexSize/2+1) : rand.nextInt(indexSize*2);
+      }
+      params.add("facet.limit", Integer.toString(limit));
+    }
+
+    if (rand.nextBoolean()) {
+      params.add("facet.sort", rand.nextBoolean() ? "index" : "count");
+    }
+
+    if ( rand.nextInt(100) < 20) {
+      final String[] prefixes = new String[] {"A","B","C"};
+      params.add("facet.prefix", prefixes[rand.nextInt(prefixes.length)]);
+    }
+
+    if (rand.nextInt(100) < 20) {
+      params.add("facet.missing", "true");
+    }
+    
+    if (rand.nextInt(100) < 20) { // assigning only valid vals
+      params.add("facet.mincount", rand.nextBoolean() ? "0": "1" );
+    }
+    
+    query(params);
+  }
+  
+  private void checkInvalidMincount() throws SolrServerException, IOException {
+    final ModifiableSolrParams params = buildParams();
+    if (random().nextBoolean()) {
+      params.remove("facet.exists");
+      params.set("f."+FLD+".facet.exists","true");
+    }
+    
+    if (random().nextBoolean()) {
+      params.set("facet.mincount",  ""+(2+random().nextInt(100)) );
+    } else {
+      params.set("f."+FLD+".facet.mincount",  ""+(2+random().nextInt(100)) );
+    }
+    
+    try {
+      if (random().nextBoolean()) {
+        setDistributedParams(params);
+        queryServer(params);
+      } else {
+        params.set("distrib", "false");
+        controlClient.query(params);
+      }
+      fail();
+    } catch(SolrException e) { // check that distr and single index search fail the same
+      assertEquals(e.code(), ErrorCode.BAD_REQUEST.code);
+      assertTrue(e.getMessage().contains("facet.exists"));
+      assertTrue(e.getMessage().contains("facet.mincount"));
+      assertTrue(e.getMessage().contains(FLD));
+    }
+  }
+
+  private void checkBasicRequest() throws Exception {
+    final ModifiableSolrParams params = buildParams();
+    QueryResponse rsp = query(params);
+    assertResponse(rsp);
+  }
+
+  private void checkWithMinCountEqOne() throws Exception {
+    final ModifiableSolrParams params = buildParams("facet.mincount","1");
+    QueryResponse rsp = query(params);
+    assertResponse(rsp);
+  }
+
+  private void checkWithSortCount() throws Exception {
+    final ModifiableSolrParams params = buildParams("facet.sort","count");
+    QueryResponse rsp = query(params);
+    assertResponse(rsp);
+  }
+
+  private void checkWithMethodSetPerField() throws Exception {
+    final ModifiableSolrParams params = buildParams("f." + FLD + ".facet.exists", "true");
+    params.remove("facet.exists");
+    QueryResponse rsp = query(params);
+    assertResponse(rsp);
+  }
+
+  private ModifiableSolrParams buildParams(String... additionalParams) {
+    final ModifiableSolrParams params = new ModifiableSolrParams();
+
+    params.add("q", "*:*");
+    params.add("rows", "0");
+    //params.add("debugQuery", "true");
+    params.add("facet", "true");
+    params.add("sort", "id asc");
+    
+    if(random().nextBoolean()){
+      params.add("facet.method", "enum");
+    }
+    
+    params.add("facet.exists", "true");
+    params.add("facet.field", FLD);
+    for(int i = 0; i < additionalParams.length;) {
+      params.add(additionalParams[i++], additionalParams[i++]);
+    }
+    return params;
+  }
+
+  private void assertResponse(QueryResponse rsp) {
+    final FacetField facetField = rsp.getFacetField(FLD);
+
+    assertThat(facetField.getValueCount(), is(6));
+    final List<FacetField.Count> counts = facetField.getValues();
+    for (FacetField.Count count : counts) {
+      assertThat("Count for: " + count.getName(), count.getCount(), is(1L));
+    }
+    assertThat(counts.get(0).getName(), is("AAA"));
+    assertThat(counts.get(1).getName(), is("B"));
+    assertThat(counts.get(2).getName(), is("BB"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ff69d148/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java b/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java
index 0514918..85035b9 100644
--- a/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java
+++ b/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java
@@ -38,7 +38,6 @@ import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.util.TimeZoneUtils;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.noggit.ObjectBuilder;
 import org.slf4j.Logger;
@@ -494,11 +493,9 @@ public class SimpleFacetsTest extends SolrTestCaseJ4 {
 
     ModifiableSolrParams params = params("q","*:*", "rows","0", "facet","true", "facet.field","{!key=myalias}"+field);
     
-    String[] methods = {null, "fc","enum","fcs", "uif"
-        };
+    String[] methods = {null, "fc","enum","fcs", "uif"};
     if (sf.multiValued() || sf.getType().multiValuedFieldCache()) {
-      methods = new String[]{null, "fc","enum", "uif"
-          };
+      methods = new String[]{null, "fc","enum", "uif"};
     }
 
     prefixes = prefixes==null ? new String[]{null} : prefixes;
@@ -2017,6 +2014,49 @@ public class SimpleFacetsTest extends SolrTestCaseJ4 {
     doFacetPrefix("t_s", null, "", "facet.method", "enum", "facet.enum.cache.minDf", "3");
     doFacetPrefix("t_s", null, "", "facet.method", "enum", "facet.enum.cache.minDf", "100");
     doFacetPrefix("t_s", null, "", "facet.method", "fc");
+    doFacetExistsPrefix("t_s", null, "");
+    doFacetExistsPrefix("t_s", null, "", "facet.enum.cache.minDf", "3");
+    doFacetExistsPrefix("t_s", null, "", "facet.enum.cache.minDf", "100");
+  }
+
+  @Test
+  public void testFacetExistsShouldThrowExceptionForMincountGreaterThanOne () throws Exception {
+    final String f = "t_s";
+    final List<String> msg = Arrays.asList("facet.mincount", "facet.exists", f);
+    Collections.shuffle(msg, random());
+    assertQEx("checking global method or per field", msg.get(0), 
+        req("q", "id:[* TO *]"
+            ,"indent","on"
+            ,"facet","true"
+            , random().nextBoolean() ? "facet.exists": "f."+f+".facet.exists", "true"
+            ,"facet.field", f
+            , random().nextBoolean() ? "facet.mincount" : "f."+f+".facet.mincount" ,
+                 "" + (2+random().nextInt(Integer.MAX_VALUE-2))
+        )
+        , ErrorCode.BAD_REQUEST);
+    
+    assertQ("overriding per field",
+        req("q", "id:[* TO *]"
+            ,"indent","on"
+            ,"facet","true"
+            ,"facet.exists", "true"
+            ,"f."+f+".facet.exists", "false"
+            ,"facet.field", f
+            ,"facet.mincount",""+(2+random().nextInt(Integer.MAX_VALUE-2))
+        ),
+        "//lst[@name='facet_fields']/lst[@name='"+f+"']");
+    
+    assertQ("overriding per field",
+        req("q", "id:[* TO *]"
+            ,"indent","on"
+            ,"facet","true"
+            ,"facet.exists", "true"
+            ,"facet.field", f
+            ,"facet.mincount",""+(2+random().nextInt(Integer.MAX_VALUE-2))
+            ,"f."+f+".facet.mincount", random().nextBoolean() ? "0":"1"
+        ),
+        "//lst[@name='facet_fields']/lst[@name='"+f+"']");
+    
   }
 
   static void indexFacetPrefixSingleValued() {
@@ -2037,7 +2077,7 @@ public class SimpleFacetsTest extends SolrTestCaseJ4 {
   }
   
   @Test
-  @Ignore("SOLR-8466 - facet.method=uif ignores facet.contains")
+  //@Ignore("SOLR-8466 - facet.method=uif ignores facet.contains")
   public void testFacetContainsUif() {
     doFacetContains("contains_s1", "contains_group_s1", "Astra", "BAst", "Ast", "facet.method", "uif");
     doFacetPrefix("contains_s1", null, "Astra", "facet.method", "uif", "facet.contains", "Ast");
@@ -2063,6 +2103,7 @@ public class SimpleFacetsTest extends SolrTestCaseJ4 {
     doFacetPrefix("contains_s1", null, "Astra", "facet.method", "enum", "facet.contains", "aSt", "facet.contains.ignoreCase", "true");
     doFacetPrefix("contains_s1", null, "Astra", "facet.method", "fcs", "facet.contains", "asT", "facet.contains.ignoreCase", "true");
     doFacetPrefix("contains_s1", null, "Astra", "facet.method", "fc", "facet.contains", "aST", "facet.contains.ignoreCase", "true");
+    doFacetExistsPrefix("contains_s1", null, "Astra", "facet.contains", "Ast");
   }
 
   static void indexFacetPrefix(String idPrefix, String f, String termSuffix, String g) {
@@ -2313,6 +2354,239 @@ public class SimpleFacetsTest extends SolrTestCaseJ4 {
     );
   }
 
+  public void doFacetExistsPrefix(String f, String local, String termSuffix, String... params) {
+    String indent="on";
+    String pre = "//lst[@name='"+f+"']";
+    String lf = local==null ? f : local+f;
+
+    assertQ("test field facet.method",
+        req(params, "q", "id:[* TO *]"
+            ,"indent", indent
+            ,"facet", "true"
+            ,"f."+lf+".facet.exists", "true"
+            ,"facet.field", lf
+            ,"facet.mincount", "0"
+            ,"facet.offset", "0"
+            ,"facet.limit", "100"
+            ,"facet.sort", "count"
+            ,"facet.prefix", "B"
+        )
+        ,"*[count(//lst[@name='facet_fields']/lst/int)=3]"
+        ,pre+"/int[1][@name='B"+termSuffix+"'][.='1']"
+        ,pre+"/int[2][@name='BB"+termSuffix+"'][.='1']"
+        ,pre+"/int[3][@name='BBB"+termSuffix+"'][.='1']"
+    );
+
+    assertQ("test facet.prefix middle, exact match first term",
+            req(params, "q", "id:[* TO *]"
+                    ,"indent",indent
+                    ,"facet","true"
+                    ,"facet.exists", "true"
+                    ,"facet.field", lf
+                    ,"facet.mincount","0"
+                    ,"facet.offset","0"
+                    ,"facet.limit","100"
+                    ,"facet.sort","count"
+                    ,"facet.prefix","B"
+            )
+            ,"*[count(//lst[@name='facet_fields']/lst/int)=3]"
+            ,pre+"/int[1][@name='B"+termSuffix+"'][.='1']"
+            ,pre+"/int[2][@name='BB"+termSuffix+"'][.='1']"
+            ,pre+"/int[3][@name='BBB"+termSuffix+"'][.='1']"
+    );
+
+    assertQ("test facet.prefix middle, exact match first term, unsorted",
+            req(params, "q", "id:[* TO *]"
+                    ,"indent",indent
+                    ,"facet","true"
+                    ,"facet.exists", "true"
+                    ,"facet.field", lf
+                    ,"facet.mincount","0"
+                    ,"facet.offset","0"
+                    ,"facet.limit","100"
+                    ,"facet.sort","index"
+                    ,"facet.prefix","B"
+            )
+            ,"*[count(//lst[@name='facet_fields']/lst/int)=3]"
+            ,pre+"/int[1][@name='B"+termSuffix+"'][.='1']"
+            ,pre+"/int[2][@name='BB"+termSuffix+"'][.='1']"
+            ,pre+"/int[3][@name='BBB"+termSuffix+"'][.='1']"
+    );
+
+    assertQ("test facet.prefix middle, paging",
+            req(params, "q", "id:[* TO *]"
+                    ,"indent",indent
+                    ,"facet","true"
+                    ,"facet.exists", "true"
+                    ,"facet.field", lf
+                    ,"facet.mincount","0"
+                    ,"facet.offset","1"
+                    ,"facet.limit","100"
+                    ,"facet.sort","count"
+                    ,"facet.prefix","B"
+            )
+            ,"*[count(//lst[@name='facet_fields']/lst/int)=2]"
+            ,pre+"/int[1][@name='BB"+termSuffix+"'][.='1']"
+            ,pre+"/int[2][@name='BBB"+termSuffix+"'][.='1']"
+    );
+
+    assertQ("test facet.prefix middle, paging",
+            req(params, "q", "id:[* TO *]"
+                    ,"indent",indent
+                    ,"facet","true"
+                    ,"facet.exists", "true"
+                    ,"facet.field", lf
+                    ,"facet.mincount","0"
+                    ,"facet.offset","1"
+                    ,"facet.limit","1"
+                    ,"facet.sort","count"
+                    ,"facet.prefix","B"
+            )
+            ,"*[count(//lst[@name='facet_fields']/lst/int)=1]"
+            ,pre+"/int[1][@name='BB"+termSuffix+"'][.='1']"
+    );
+
+    assertQ("test facet.prefix end, not exact match",
+            req(params, "q", "id:[* TO *]"
+                    ,"indent",indent
+                    ,"facet","true"
+                    ,"facet.exists", "true"
+                    ,"facet.field", lf
+                    ,"facet.mincount","0"
+                    ,"facet.offset","0"
+                    ,"facet.limit","100"
+                    ,"facet.sort","count"
+                    ,"facet.prefix","C"
+            )
+            ,"*[count(//lst[@name='facet_fields']/lst/int)=2]"
+            ,pre+"/int[1][@name='CC"+termSuffix+"'][.='1']"
+            ,pre+"/int[2][@name='CCC"+termSuffix+"'][.='1']"
+    );
+
+    assertQ("test facet.prefix end, exact match",
+            req(params, "q", "id:[* TO *]"
+                    ,"indent",indent
+                    ,"facet","true"
+                    ,"facet.exists", "true"
+                    ,"facet.field", lf
+                    ,"facet.mincount","0"
+                    ,"facet.offset","0"
+                    ,"facet.limit","100"
+                    ,"facet.sort","count"
+                    ,"facet.prefix","CC"
+            )
+            ,"*[count(//lst[@name='facet_fields']/lst/int)=2]"
+            ,pre+"/int[1][@name='CC"+termSuffix+"'][.='1']"
+            ,pre+"/int[2][@name='CCC"+termSuffix+"'][.='1']"
+    );
+
+    assertQ("test facet.prefix past end",
+            req(params, "q", "id:[* TO *]"
+                    ,"indent",indent
+                    ,"facet","true"
+                    ,"facet.exists", "true"
+                    ,"facet.field", lf
+                    ,"facet.mincount","0"
+                    ,"facet.offset","0"
+                    ,"facet.limit","100"
+                    ,"facet.sort","count"
+                    ,"facet.prefix","X"
+            )
+            ,"*[count(//lst[@name='facet_fields']/lst/int)=0]"
+    );
+
+    assertQ("test facet.prefix past end",
+            req(params, "q", "id:[* TO *]"
+                    ,"indent",indent
+                    ,"facet","true"
+                    ,"facet.exists", "true"
+                    ,"facet.field", lf
+                    ,"facet.mincount","0"
+                    ,"facet.offset","1"
+                    ,"facet.limit","-1"
+                    ,"facet.sort","count"
+                    ,"facet.prefix","X"
+            )
+            ,"*[count(//lst[@name='facet_fields']/lst/int)=0]"
+    );
+
+    assertQ("test facet.prefix at start, exact match",
+            req(params, "q", "id:[* TO *]"
+                    ,"indent",indent
+                    ,"facet","true"
+                    ,"facet.exists", "true"
+                    ,"facet.field", lf
+                    ,"facet.mincount","0"
+                    ,"facet.offset","0"
+                    ,"facet.limit","100"
+                    ,"facet.sort","count"
+                    ,"facet.prefix","AAA"
+            )
+            ,"*[count(//lst[@name='facet_fields']/lst/int)=1]"
+            ,pre+"/int[1][@name='AAA"+termSuffix+"'][.='1']"
+    );
+    assertQ("test facet.prefix at Start, not exact match",
+            req(params, "q", "id:[* TO *]"
+                    ,"indent",indent
+                    ,"facet","true"
+                    ,"facet.exists", "true"
+                    ,"facet.field", lf
+                    ,"facet.mincount","0"
+                    ,"facet.offset","0"
+                    ,"facet.limit","100"
+                    ,"facet.sort","count"
+                    ,"facet.prefix","AA"
+            )
+            ,"*[count(//lst[@name='facet_fields']/lst/int)=1]"
+            ,pre+"/int[1][@name='AAA"+termSuffix+"'][.='1']"
+    );
+    assertQ("test facet.prefix before start",
+            req(params, "q", "id:[* TO *]"
+                    ,"indent",indent
+                    ,"facet","true"
+                    ,"facet.exists", "true"
+                    ,"facet.field", lf
+                    ,"facet.mincount","0"
+                    ,"facet.offset","0"
+                    ,"facet.limit","100"
+                    ,"facet.sort","count"
+                    ,"facet.prefix","999"
+            )
+            ,"*[count(//lst[@name='facet_fields']/lst/int)=0]"
+    );
+
+    assertQ("test facet.prefix before start",
+            req(params, "q", "id:[* TO *]"
+                    ,"indent",indent
+                    ,"facet","true"
+                    ,"facet.exists", "true"
+                    ,"facet.field", lf
+                    ,"facet.mincount","0"
+                    ,"facet.offset","2"
+                    ,"facet.limit","100"
+                    ,"facet.sort","count"
+                    ,"facet.prefix","999"
+            )
+            ,"*[count(//lst[@name='facet_fields']/lst/int)=0]"
+    );
+
+    // test offset beyond what is collected internally in queue
+    assertQ(
+            req(params, "q", "id:[* TO *]"
+                    ,"indent",indent
+                    ,"facet","true"
+                    ,"facet.exists", "true"
+                    ,"facet.field", lf
+                    ,"facet.mincount","1"
+                    ,"facet.offset","5"
+                    ,"facet.limit","10"
+                    ,"facet.sort","count"
+                    ,"facet.prefix","CC"
+            )
+            ,"*[count(//lst[@name='facet_fields']/lst/int)=0]"
+    );
+  }
+
   public void doFacetContains(String f, String g, String termSuffix, String contains, String groupContains, String... params) {
     String indent="on";
     String pre = "//lst[@name='"+f+"']";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ff69d148/solr/solrj/src/java/org/apache/solr/common/params/FacetParams.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/FacetParams.java b/solr/solrj/src/java/org/apache/solr/common/params/FacetParams.java
index e014c86..038fc6e 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/FacetParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/FacetParams.java
@@ -185,6 +185,14 @@ public interface FacetParams {
    * only use the filterCache for terms with a df &gt;= to this parameter.
    */
   public static final String FACET_ENUM_CACHE_MINDF = FACET + ".enum.cache.minDf";
+  
+  /**
+   *  A boolean parameter that caps the facet counts at 1. 
+   *  With this set, a returned count will only be 0 or 1. 
+   *  For apps that don't need the count, this should be an optimization
+   */
+  public static final String FACET_EXISTS = FACET+".exists";
+  
   /**
    * Any field whose terms the user wants to enumerate over for
    * Facet Contraint Counts (multi-value)