You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by tf...@apache.org on 2020/05/08 21:26:01 UTC

[lucene-solr] branch master updated: SOLR-13289: Add Support for BlockMax WAND (#1456)

This is an automated email from the ASF dual-hosted git repository.

tflobbe pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new d9f9d6d  SOLR-13289: Add Support for BlockMax WAND (#1456)
d9f9d6d is described below

commit d9f9d6dd47c06f5fe092d43d6bf0c77c5ff2019f
Author: Tomas Fernandez Lobbe <tf...@apache.org>
AuthorDate: Fri May 8 14:25:47 2020 -0700

    SOLR-13289: Add Support for BlockMax WAND (#1456)
    
    Add support for BlockMax WAND via a minExactHits parameter. Hits will be counted accurately at least until this value, and above that, the count will be an approximation. In distributed search requests, the count will be per shard, so potentially the count will be accurately counted until numShards * minExactHits. The response will include the value numFoundExact which can be true (The value in numFound is exact) or false (the value in numFound is an approximation).
---
 solr/CHANGES.txt                                   |   5 +
 .../clustering/carrot2/CarrotClusteringEngine.java |   3 +-
 .../client/solrj/embedded/EmbeddedSolrServer.java  |   2 +
 .../solr/handler/component/ExpandComponent.java    |   4 +-
 .../solr/handler/component/QueryComponent.java     |  19 +-
 .../solr/handler/component/ResponseBuilder.java    |   3 +-
 .../solr/handler/component/SearchHandler.java      |   2 +
 .../solr/handler/tagger/TaggerRequestHandler.java  |   3 +-
 .../apache/solr/response/BinaryResponseWriter.java |   4 +-
 .../solr/response/GeoJSONResponseWriter.java       |  18 +-
 .../apache/solr/response/JSONResponseWriter.java   |   8 +
 .../java/org/apache/solr/response/JSONWriter.java  |  43 ++++-
 .../solr/response/PHPSerializedResponseWriter.java |  16 +-
 .../org/apache/solr/response/SchemaXmlWriter.java  |   5 +
 .../solr/response/TabularResponseWriter.java       |   7 +
 .../apache/solr/response/TextResponseWriter.java   |  16 +-
 .../java/org/apache/solr/response/XMLWriter.java   |  22 +++
 .../transform/SubQueryAugmenterFactory.java        |   4 +-
 .../src/java/org/apache/solr/search/DocList.java   |   3 +
 .../src/java/org/apache/solr/search/DocSlice.java  |  13 +-
 .../src/java/org/apache/solr/search/Grouping.java  |   6 +-
 .../org/apache/solr/search/MaxScoreCollector.java  |   4 +-
 .../java/org/apache/solr/search/QueryCommand.java  |  10 ++
 .../org/apache/solr/search/QueryResultKey.java     |  15 +-
 .../org/apache/solr/search/SolrIndexSearcher.java  |  24 +--
 .../org/apache/solr/BasicFunctionalityTest.java    |   2 +-
 .../test/org/apache/solr/TestCrossCoreJoin.java    |   4 +-
 .../org/apache/solr/TestDistributedSearch.java     |  32 +++-
 .../test/org/apache/solr/TestGroupingSearch.java   | 184 +++++++++++--------
 solr/core/src/test/org/apache/solr/TestJoin.java   |  19 +-
 .../org/apache/solr/core/QueryResultKeyTest.java   |  16 +-
 .../org/apache/solr/handler/tagger/TaggerTest.java |   6 +-
 .../test/org/apache/solr/request/TestFaceting.java |  25 +++
 .../org/apache/solr/response/JSONWriterTest.java   |   4 +-
 .../response/TestPHPSerializedResponseWriter.java  |   2 +-
 .../apache/solr/search/SolrIndexSearcherTest.java  | 200 +++++++++++++++++++++
 .../solr/search/TestAddFieldRealTimeGet.java       |   8 +-
 .../test/org/apache/solr/search/TestDocSet.java    |   3 +-
 .../org/apache/solr/search/TestRealTimeGet.java    |   6 +-
 .../apache/solr/search/facet/TestJsonFacets.java   |   2 +-
 .../facet/TestJsonFacetsWithNestedObjects.java     |  12 +-
 .../solr/search/join/TestScoreJoinQPNoScore.java   |  23 +--
 .../solr/search/join/TestScoreJoinQPScore.java     |  18 +-
 .../org/apache/solr/common/SolrDocumentList.java   |  10 ++
 .../apache/solr/common/params/CommonParams.java    |   6 +
 .../solr/common/util/FastJavaBinDecoder.java       |   3 +
 .../org/apache/solr/common/util/JavaBinCodec.java  |   9 +-
 .../src/test-files/solrj/javabin_backcompat.bin    | Bin 169 -> 170 bytes
 .../solr/common/params/CommonParamsTest.java       |   2 +
 .../apache/solr/common/util/TestJavaBinCodec.java  |  45 ++---
 50 files changed, 704 insertions(+), 196 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index fb65169..99fce5b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -122,6 +122,11 @@ Optimizations
 
 * LUCENE-7788: fail precommit on unparameterised log messages and examine for wasted work/objects (Erick Erickson)
 
+* SOLR-13289: When the "minExactHits" parameters is provided in queries and it's value is lower than the number of hits,
+  Solr can speedup the query resolution by using the Block-Max WAND algorithm (see LUCENE-8135). When doing this, the
+  value of matching documents in the response (numFound) will be an approximation.
+  (Ishan Chattopadhyaya, Munendra S N, Tomás Fernández Löbbe)
+
 Bug Fixes
 ---------------------
 * SOLR-13264: IndexSizeTrigger aboveOp / belowOp properties not in valid properties.
diff --git a/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/CarrotClusteringEngine.java b/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/CarrotClusteringEngine.java
index 0e4628d..8e1f625 100644
--- a/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/CarrotClusteringEngine.java
+++ b/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/CarrotClusteringEngine.java
@@ -33,6 +33,7 @@ import java.util.function.Supplier;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TotalHits;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.SolrException;
@@ -358,7 +359,7 @@ public class CarrotClusteringEngine extends SearchClusteringEngine {
       // See comment in ClusteringComponent#finishStage().
       if (produceSummary && docIds != null) {
         docsHolder[0] = docIds.get(sdoc).intValue();
-        DocList docAsList = new DocSlice(0, 1, docsHolder, scores, 1, 1.0f);
+        DocList docAsList = new DocSlice(0, 1, docsHolder, scores, 1, 1.0f, TotalHits.Relation.EQUAL_TO);
         NamedList<Object> highlights = highlighter.doHighlighting(docAsList, theQuery, req, snippetFieldAry);
         if (highlights != null && highlights.size() == 1) {
           // should only be one value given our setup
diff --git a/solr/core/src/java/org/apache/solr/client/solrj/embedded/EmbeddedSolrServer.java b/solr/core/src/java/org/apache/solr/client/solrj/embedded/EmbeddedSolrServer.java
index 1b4d962..bbbb8a0 100644
--- a/solr/core/src/java/org/apache/solr/client/solrj/embedded/EmbeddedSolrServer.java
+++ b/solr/core/src/java/org/apache/solr/client/solrj/embedded/EmbeddedSolrServer.java
@@ -30,6 +30,7 @@ import java.util.Set;
 import java.util.function.Supplier;
 
 import org.apache.commons.io.output.ByteArrayOutputStream;
+import org.apache.lucene.search.TotalHits.Relation;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
@@ -238,6 +239,7 @@ public class EmbeddedSolrServer extends SolrClient {
                   // write an empty list...
                   SolrDocumentList docs = new SolrDocumentList();
                   docs.setNumFound(ctx.getDocList().matches());
+                  docs.setNumFoundExact(ctx.getDocList().hitCountRelation() == Relation.EQUAL_TO);
                   docs.setStart(ctx.getDocList().offset());
                   docs.setMaxScore(ctx.getDocList().maxScore());
                   codec.writeSolrDocumentList(docs);
diff --git a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
index 5e7e5ea..84f21b6 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
@@ -439,13 +439,13 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
             scores[i] = scoreDoc.score;
           }
           assert topDocs.totalHits.relation == TotalHits.Relation.EQUAL_TO;
-          DocSlice slice = new DocSlice(0, docs.length, docs, scores, topDocs.totalHits.value, Float.NaN);
+          DocSlice slice = new DocSlice(0, docs.length, docs, scores, topDocs.totalHits.value, Float.NaN, TotalHits.Relation.EQUAL_TO);
           addGroupSliceToOutputMap(fieldType, ordBytes, outMap, charsRef, groupValue, slice);
         }
       } else {
         int totalHits = ((TotalHitCountCollector) cursor.value).getTotalHits();
         if (totalHits > 0) {
-          DocSlice slice = new DocSlice(0, 0, null, null, totalHits, 0);
+          DocSlice slice = new DocSlice(0, 0, null, null, totalHits, 0, TotalHits.Relation.EQUAL_TO);
           addGroupSliceToOutputMap(fieldType, ordBytes, outMap, charsRef, groupValue, slice);
         }
       }
diff --git a/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java b/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
index 9f8b355..c113bef 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
@@ -44,6 +44,7 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorable;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.TotalHits;
 import org.apache.lucene.search.grouping.GroupDocs;
 import org.apache.lucene.search.grouping.SearchGroup;
 import org.apache.lucene.search.grouping.TopGroups;
@@ -365,6 +366,7 @@ public class QueryComponent extends SearchComponent
 
     QueryCommand cmd = rb.createQueryCommand();
     cmd.setTimeAllowed(timeAllowed);
+    cmd.setMinExactHits(getMinExactHits(params));
 
     req.getContext().put(SolrIndexSearcher.STATS_SOURCE, statsCache.get(req));
     
@@ -401,6 +403,14 @@ public class QueryComponent extends SearchComponent
     doProcessUngroupedSearch(rb, cmd, result);
   }
 
+  private int getMinExactHits(SolrParams params) {
+    long minExactHits = params.getLong(CommonParams.MIN_EXACT_HITS, Integer.MAX_VALUE);
+    if (minExactHits < 0 || minExactHits > Integer.MAX_VALUE) {
+      minExactHits = Integer.MAX_VALUE;
+    }
+    return (int)minExactHits;
+  }
+
   protected void doFieldSortValues(ResponseBuilder rb, SolrIndexSearcher searcher) throws IOException
   {
     SolrQueryRequest req = rb.req;
@@ -840,6 +850,7 @@ public class QueryComponent extends SearchComponent
       }
       
       long numFound = 0;
+      boolean hitCountIsExact = true;
       Float maxScore=null;
       boolean thereArePartialResults = false;
       Boolean segmentTerminatedEarly = null;
@@ -871,6 +882,7 @@ public class QueryComponent extends SearchComponent
             }
             docs = (SolrDocumentList)srsp.getSolrResponse().getResponse().get("response");
             nl.add("numFound", docs.getNumFound());
+            nl.add("numFoundExact", docs.getNumFoundExact());
             nl.add("maxScore", docs.getMaxScore());
             nl.add("shardAddress", srsp.getShardAddress());
           }
@@ -912,6 +924,10 @@ public class QueryComponent extends SearchComponent
           maxScore = maxScore==null ? docs.getMaxScore() : Math.max(maxScore, docs.getMaxScore());
         }
         numFound += docs.getNumFound();
+        
+        if (hitCountIsExact && Boolean.FALSE.equals(docs.getNumFoundExact())) {
+          hitCountIsExact = false;
+        }
 
         NamedList sortFieldValues = (NamedList)(srsp.getSolrResponse().getResponse().get("sort_values"));
         if (sortFieldValues.size()==0 && // we bypass merging this response only if it's partial itself
@@ -983,6 +999,7 @@ public class QueryComponent extends SearchComponent
       SolrDocumentList responseDocs = new SolrDocumentList();
       if (maxScore!=null) responseDocs.setMaxScore(maxScore);
       responseDocs.setNumFound(numFound);
+      responseDocs.setNumFoundExact(hitCountIsExact);
       responseDocs.setStart(ss.getOffset());
       // size appropriately
       for (int i=0; i<resultSize; i++) responseDocs.add(null);
@@ -1274,7 +1291,7 @@ public class QueryComponent extends SearchComponent
     }
 
     DocListAndSet res = new DocListAndSet();
-    res.docList = new DocSlice(0, docs, luceneIds, null, docs, 0);
+    res.docList = new DocSlice(0, docs, luceneIds, null, docs, 0, TotalHits.Relation.EQUAL_TO);
     if (rb.isNeedDocSet()) {
       // TODO: create a cache for this!
       List<Query> queries = new ArrayList<>();
diff --git a/solr/core/src/java/org/apache/solr/handler/component/ResponseBuilder.java b/solr/core/src/java/org/apache/solr/handler/component/ResponseBuilder.java
index 40af722..ce51dec 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/ResponseBuilder.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/ResponseBuilder.java
@@ -24,6 +24,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TotalHits;
 import org.apache.lucene.search.grouping.SearchGroup;
 import org.apache.lucene.search.grouping.TopGroups;
 import org.apache.lucene.util.BytesRef;
@@ -450,7 +451,7 @@ public class ResponseBuilder
       rsp.getResponseHeader().asShallowMap()
           .put(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY, Boolean.TRUE);
       if(getResults() != null && getResults().docList==null) {
-        getResults().docList = new DocSlice(0, 0, new int[] {}, new float[] {}, 0, 0);
+        getResults().docList = new DocSlice(0, 0, new int[] {}, new float[] {}, 0, 0, TotalHits.Relation.EQUAL_TO);
       }
     }
     final Boolean segmentTerminatedEarly = result.getSegmentTerminatedEarly();
diff --git a/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java b/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
index 16e39fb..634454be 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
@@ -26,6 +26,7 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.lucene.index.ExitableDirectoryReader;
+import org.apache.lucene.search.TotalHits;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrDocumentList;
@@ -484,6 +485,7 @@ public class SearchHandler extends RequestHandlerBase implements SolrCoreAware,
       }
       else {
         nl.add("numFound", rb.getResults().docList.matches());
+        nl.add("numFoundExact", rb.getResults().docList.hitCountRelation() == TotalHits.Relation.EQUAL_TO);
         nl.add("maxScore", rb.getResults().docList.maxScore());
       }
       nl.add("shardAddress", rb.shortCircuitedURL);
diff --git a/solr/core/src/java/org/apache/solr/handler/tagger/TaggerRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/tagger/TaggerRequestHandler.java
index adc8947..42d1ad6 100644
--- a/solr/core/src/java/org/apache/solr/handler/tagger/TaggerRequestHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/tagger/TaggerRequestHandler.java
@@ -47,6 +47,7 @@ import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TotalHits;
 import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.FixedBitSet;
@@ -284,7 +285,7 @@ public class TaggerRequestHandler extends RequestHandlerBase {
     for (int i = 0; i < docIds.length; i++) {
       docIds[i] = docIdIter.nextDoc();
     }
-    return new DocSlice(0, docIds.length, docIds, null, matchDocs, 1f);
+    return new DocSlice(0, docIds.length, docIds, null, matchDocs, 1f, TotalHits.Relation.EQUAL_TO);
   }
 
   private TagClusterReducer chooseTagClusterReducer(String overlaps) {
diff --git a/solr/core/src/java/org/apache/solr/response/BinaryResponseWriter.java b/solr/core/src/java/org/apache/solr/response/BinaryResponseWriter.java
index 33d1541..617f92c 100644
--- a/solr/core/src/java/org/apache/solr/response/BinaryResponseWriter.java
+++ b/solr/core/src/java/org/apache/solr/response/BinaryResponseWriter.java
@@ -31,6 +31,7 @@ import java.util.function.Consumer;
 import org.apache.commons.io.output.ByteArrayOutputStream;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.search.TotalHits;
 import org.apache.solr.client.solrj.impl.BinaryResponseParser;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.params.CommonParams;
@@ -164,7 +165,7 @@ public class BinaryResponseWriter implements BinaryQueryResponseWriter {
 
     public void writeResults(ResultContext ctx, JavaBinCodec codec) throws IOException {
       codec.writeTag(JavaBinCodec.SOLRDOCLST);
-      List l = new ArrayList(3);
+      List<Object> l = new ArrayList<>(4);
       l.add((long) ctx.getDocList().matches());
       l.add((long) ctx.getDocList().offset());
       
@@ -173,6 +174,7 @@ public class BinaryResponseWriter implements BinaryQueryResponseWriter {
         maxScore = ctx.getDocList().maxScore();
       }
       l.add(maxScore);
+      l.add(ctx.getDocList().hitCountRelation() == TotalHits.Relation.EQUAL_TO);
       codec.writeArray(l);
       
       // this is a seprate function so that streaming responses can use just that part
diff --git a/solr/core/src/java/org/apache/solr/response/GeoJSONResponseWriter.java b/solr/core/src/java/org/apache/solr/response/GeoJSONResponseWriter.java
index 90b20ff..422ad0d 100644
--- a/solr/core/src/java/org/apache/solr/response/GeoJSONResponseWriter.java
+++ b/solr/core/src/java/org/apache/solr/response/GeoJSONResponseWriter.java
@@ -292,11 +292,18 @@ class GeoJSONWriter extends JSONWriter {
     }
   }
 
+  @Deprecated
   @Override
   public void writeStartDocumentList(String name, 
-      long start, int size, long numFound, Float maxScore) throws IOException
+      long start, int size, long numFound, Float maxScore) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  @Override
+  public void writeStartDocumentList(String name, 
+      long start, int size, long numFound, Float maxScore, Boolean numFoundExact) throws IOException
   {
-    writeMapOpener((maxScore==null) ? 3 : 4);
+    writeMapOpener(headerSize(maxScore, numFoundExact));
     incLevel();
     writeKey("type",false);
     writeStr(null, "FeatureCollection", false);
@@ -312,6 +319,13 @@ class GeoJSONWriter extends JSONWriter {
       writeKey("maxScore",false);
       writeFloat(null,maxScore);
     }
+    
+    if (numFoundExact != null) {
+      writeMapSeparator();
+      writeKey("numFoundExact",false);
+      writeBool(null, numFoundExact);
+    }
+    
     writeMapSeparator();
     
     // if can we get bbox of all results, we should write it here
diff --git a/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java b/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java
index a52aef6..596a05e 100644
--- a/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java
+++ b/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java
@@ -215,11 +215,19 @@ class ArrayOfNameTypeValueJSONWriter extends JSONWriter {
     super.writeSolrDocument(name, doc, returnFields, idx);
   }
 
+  @Deprecated
   @Override
   public void writeStartDocumentList(String name, long start, int size, long numFound, Float maxScore) throws IOException {
     ifNeededWriteTypeAndValueKey("doclist");
     super.writeStartDocumentList(name, start, size, numFound, maxScore);
   }
+  
+  @Override
+  public void writeStartDocumentList(String name, long start, int size, long numFound, Float maxScore, Boolean numFoundExact) throws IOException {
+    ifNeededWriteTypeAndValueKey("doclist");
+    super.writeStartDocumentList(name, start, size, numFound, maxScore, numFoundExact);
+  }
+
 
   @Override
   public void writeMap(String name, Map val, boolean excludeOuter, boolean isFirstVal) throws IOException {
diff --git a/solr/core/src/java/org/apache/solr/response/JSONWriter.java b/solr/core/src/java/org/apache/solr/response/JSONWriter.java
index 77aee3d..48587a9 100644
--- a/solr/core/src/java/org/apache/solr/response/JSONWriter.java
+++ b/solr/core/src/java/org/apache/solr/response/JSONWriter.java
@@ -132,11 +132,16 @@ public class JSONWriter extends TextResponseWriter implements JsonTextWriter {
   //       that the size could not be reliably determined.
   //
 
+  /**
+   * This method will be removed in Solr 9
+   * @deprecated Use {{@link #writeStartDocumentList(String, long, int, long, Float, Boolean)}.
+   */
   @Override
+  @Deprecated
   public void writeStartDocumentList(String name,
       long start, int size, long numFound, Float maxScore) throws IOException
   {
-    writeMapOpener((maxScore==null) ? 3 : 4);
+    writeMapOpener(headerSize(maxScore, null));
     incLevel();
     writeKey("numFound",false);
     writeLong(null,numFound);
@@ -156,6 +161,42 @@ public class JSONWriter extends TextResponseWriter implements JsonTextWriter {
 
     incLevel();
   }
+  
+  @Override
+  public void writeStartDocumentList(String name,
+      long start, int size, long numFound, Float maxScore, Boolean numFoundExact) throws IOException {
+    writeMapOpener(headerSize(maxScore, numFoundExact));
+    incLevel();
+    writeKey("numFound",false);
+    writeLong(null,numFound);
+    writeMapSeparator();
+    writeKey("start",false);
+    writeLong(null,start);
+
+    if (maxScore != null) {
+      writeMapSeparator();
+      writeKey("maxScore",false);
+      writeFloat(null,maxScore);
+    }
+    
+    if (numFoundExact != null) {
+      writeMapSeparator();
+      writeKey("numFoundExact",false);
+      writeBool(null, numFoundExact);
+    }
+    writeMapSeparator();
+    writeKey("docs",false);
+    writeArrayOpener(size);
+
+    incLevel();
+  } 
+
+  protected int headerSize(Float maxScore, Boolean numFoundExact) {
+    int headerSize = 3;
+    if (maxScore != null) headerSize++;
+    if (numFoundExact != null) headerSize++;
+    return headerSize;
+  }
 
   @Override
   public void writeEndDocumentList() throws IOException
diff --git a/solr/core/src/java/org/apache/solr/response/PHPSerializedResponseWriter.java b/solr/core/src/java/org/apache/solr/response/PHPSerializedResponseWriter.java
index 7e090b8..9db67ee 100644
--- a/solr/core/src/java/org/apache/solr/response/PHPSerializedResponseWriter.java
+++ b/solr/core/src/java/org/apache/solr/response/PHPSerializedResponseWriter.java
@@ -89,13 +89,19 @@ class PHPSerializedWriter extends JSONWriter {
     writeNamedListAsMapMangled(name,val);
   }
   
-  
-
+  @Deprecated
   @Override
   public void writeStartDocumentList(String name, 
       long start, int size, long numFound, Float maxScore) throws IOException
   {
-    writeMapOpener((maxScore==null) ? 3 : 4);
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void writeStartDocumentList(String name, 
+      long start, int size, long numFound, Float maxScore, Boolean numFoundExact) throws IOException
+  {
+    writeMapOpener(headerSize(maxScore, numFoundExact));
     writeKey("numFound",false);
     writeLong(null,numFound);
     writeKey("start",false);
@@ -105,6 +111,10 @@ class PHPSerializedWriter extends JSONWriter {
       writeKey("maxScore",false);
       writeFloat(null,maxScore);
     }
+    if (numFoundExact != null) {
+      writeKey("numFoundExact",false);
+      writeBool(null, numFoundExact);
+    }
     writeKey("docs",false);
     writeArrayOpener(size);
   }
diff --git a/solr/core/src/java/org/apache/solr/response/SchemaXmlWriter.java b/solr/core/src/java/org/apache/solr/response/SchemaXmlWriter.java
index 8d26e0e..29ee3b2 100644
--- a/solr/core/src/java/org/apache/solr/response/SchemaXmlWriter.java
+++ b/solr/core/src/java/org/apache/solr/response/SchemaXmlWriter.java
@@ -466,6 +466,11 @@ public class SchemaXmlWriter extends TextResponseWriter {
   public void writeStartDocumentList(String name, long start, int size, long numFound, Float maxScore) throws IOException {
     // no-op
   }
+  
+  @Override
+  public void writeStartDocumentList(String name, long start, int size, long numFound, Float maxScore, Boolean numFoundExact) throws IOException {
+    // no-op
+  }
 
   @Override
   public void writeSolrDocument(String name, SolrDocument doc, ReturnFields returnFields, int idx) throws IOException {
diff --git a/solr/core/src/java/org/apache/solr/response/TabularResponseWriter.java b/solr/core/src/java/org/apache/solr/response/TabularResponseWriter.java
index a6e9e94..b22941c 100644
--- a/solr/core/src/java/org/apache/solr/response/TabularResponseWriter.java
+++ b/solr/core/src/java/org/apache/solr/response/TabularResponseWriter.java
@@ -127,6 +127,13 @@ public abstract class TabularResponseWriter extends TextResponseWriter {
   {
     // nothing
   }
+  
+  @Override
+  public void writeStartDocumentList(String name,
+                                     long start, int size, long numFound, Float maxScore, Boolean numFoundExact) throws IOException
+  {
+    // nothing
+  }
 
   @Override
   public void writeEndDocumentList() throws IOException
diff --git a/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java b/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java
index 98ef787..803c340 100644
--- a/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java
+++ b/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java
@@ -23,6 +23,7 @@ import java.util.Iterator;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.search.TotalHits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrDocumentList;
@@ -156,7 +157,16 @@ public abstract class TextResponseWriter implements TextWriter {
   // types of formats, including those where the name may come after the value (like
   // some XML formats).
 
-  public abstract void writeStartDocumentList(String name, long start, int size, long numFound, Float maxScore) throws IOException;  
+  //TODO: Make abstract in Solr 9.0
+  public void writeStartDocumentList(String name, long start, int size, long numFound, Float maxScore, Boolean numFoundExact) throws IOException {
+    writeStartDocumentList(name, start, size, numFound, maxScore);
+  }
+  
+  /**
+   * @deprecated Use {@link #writeStartDocumentList(String, long, int, long, Float, Boolean)}
+   */
+  @Deprecated
+  public abstract void writeStartDocumentList(String name, long start, int size, long numFound, Float maxScore) throws IOException;
 
   public abstract void writeSolrDocument(String name, SolrDocument doc, ReturnFields fields, int idx) throws IOException;
   
@@ -165,7 +175,7 @@ public abstract class TextResponseWriter implements TextWriter {
   // Assume each SolrDocument is already transformed
   public final void writeSolrDocumentList(String name, SolrDocumentList docs, ReturnFields fields) throws IOException
   {
-    writeStartDocumentList(name, docs.getStart(), docs.size(), docs.getNumFound(), docs.getMaxScore() );
+    writeStartDocumentList(name, docs.getStart(), docs.size(), docs.getNumFound(), docs.getMaxScore(), docs.getNumFoundExact());
     for( int i=0; i<docs.size(); i++ ) {
       writeSolrDocument( null, docs.get(i), fields, i );
     }
@@ -177,7 +187,7 @@ public abstract class TextResponseWriter implements TextWriter {
     DocList ids = res.getDocList();
     Iterator<SolrDocument> docsStreamer = res.getProcessedDocuments();
     writeStartDocumentList(name, ids.offset(), ids.size(), ids.matches(),
-        res.wantsScores() ? ids.maxScore() : null);
+        res.wantsScores() ? ids.maxScore() : null, ids.hitCountRelation() == TotalHits.Relation.EQUAL_TO);
 
     int idx = 0;
     while (docsStreamer.hasNext()) {
diff --git a/solr/core/src/java/org/apache/solr/response/XMLWriter.java b/solr/core/src/java/org/apache/solr/response/XMLWriter.java
index 39a31f3..287663e 100644
--- a/solr/core/src/java/org/apache/solr/response/XMLWriter.java
+++ b/solr/core/src/java/org/apache/solr/response/XMLWriter.java
@@ -167,6 +167,28 @@ public class XMLWriter extends TextResponseWriter {
 
   @Override
   public void writeStartDocumentList(String name,
+      long start, int size, long numFound, Float maxScore, Boolean numFoundExact) throws IOException
+  {
+    if (doIndent) indent();
+
+    writer.write("<result");
+    writeAttr(NAME, name);
+    writeAttr("numFound",Long.toString(numFound));
+    writeAttr("start",Long.toString(start));
+    if (maxScore != null) {
+      writeAttr("maxScore",Float.toString(maxScore));
+    }
+    if (numFoundExact != null) {
+      writeAttr("numFoundExact", numFoundExact.toString());
+    }
+    writer.write(">");
+
+    incLevel();
+  }
+  
+  @Override
+  @Deprecated
+  public void writeStartDocumentList(String name,
       long start, int size, long numFound, Float maxScore) throws IOException
   {
     if (doIndent) indent();
diff --git a/solr/core/src/java/org/apache/solr/response/transform/SubQueryAugmenterFactory.java b/solr/core/src/java/org/apache/solr/response/transform/SubQueryAugmenterFactory.java
index 8784a55..bc022de 100644
--- a/solr/core/src/java/org/apache/solr/response/transform/SubQueryAugmenterFactory.java
+++ b/solr/core/src/java/org/apache/solr/response/transform/SubQueryAugmenterFactory.java
@@ -24,6 +24,7 @@ import java.util.concurrent.Callable;
 
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TotalHits;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer;
 import org.apache.solr.client.solrj.request.QueryRequest;
@@ -177,7 +178,8 @@ class SubQueryAugmenter extends DocTransformer {
       return new DocSlice((int)docList.getStart(), 
           docList.size(), new int[0], new float[docList.size()],
           (int) docList.getNumFound(), 
-          docList.getMaxScore() == null ?  Float.NaN : docList.getMaxScore());
+          docList.getMaxScore() == null ?  Float.NaN : docList.getMaxScore(),
+              docList.getNumFoundExact() ? TotalHits.Relation.EQUAL_TO : TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO);
     }
 
     @Override
diff --git a/solr/core/src/java/org/apache/solr/search/DocList.java b/solr/core/src/java/org/apache/solr/search/DocList.java
index b136d28..0abc243 100644
--- a/solr/core/src/java/org/apache/solr/search/DocList.java
+++ b/solr/core/src/java/org/apache/solr/search/DocList.java
@@ -16,6 +16,7 @@
  */
 package org.apache.solr.search;
 
+import org.apache.lucene.search.TotalHits;
 
 /**
  * <code>DocList</code> represents the result of a query: an ordered list of document ids with optional score.
@@ -45,6 +46,8 @@ public interface DocList {
    * @return number of matches for the search(query &amp; any filters)
    */
   public long matches();
+  
+  public TotalHits.Relation hitCountRelation();
 
 
   /***
diff --git a/solr/core/src/java/org/apache/solr/search/DocSlice.java b/solr/core/src/java/org/apache/solr/search/DocSlice.java
index ba8fb83..489f950 100644
--- a/solr/core/src/java/org/apache/solr/search/DocSlice.java
+++ b/solr/core/src/java/org/apache/solr/search/DocSlice.java
@@ -19,6 +19,7 @@ package org.apache.solr.search;
 import java.util.Collection;
 import java.util.Collections;
 
+import org.apache.lucene.search.TotalHits;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
@@ -37,6 +38,7 @@ public class DocSlice implements DocList, Accountable {
 
   final float[] scores;  // optional score list
   final long matches;
+  final TotalHits.Relation matchesRelation;
   final float maxScore;
   final long ramBytesUsed; // cached value
 
@@ -48,8 +50,9 @@ public class DocSlice implements DocList, Accountable {
    * @param docs    array of docids starting at position 0
    * @param scores  array of scores that corresponds to docs, may be null
    * @param matches total number of matches for the query
+   * @param matchesRelation Indicates if {@code matches} is exact or an approximation
    */
-  public DocSlice(int offset, int len, int[] docs, float[] scores, long matches, float maxScore) {
+  public DocSlice(int offset, int len, int[] docs, float[] scores, long matches, float maxScore, TotalHits.Relation matchesRelation) {
     this.offset=offset;
     this.len=len;
     this.docs=docs;
@@ -57,6 +60,7 @@ public class DocSlice implements DocList, Accountable {
     this.matches=matches;
     this.maxScore=maxScore;
     this.ramBytesUsed = BASE_RAM_BYTES_USED + (docs == null ? 0 : ((long)docs.length << 2)) + (scores == null ? 0 : ((long)scores.length<<2)+RamUsageEstimator.NUM_BYTES_ARRAY_HEADER);
+    this.matchesRelation = matchesRelation;
   }
 
   @Override
@@ -70,7 +74,7 @@ public class DocSlice implements DocList, Accountable {
     int realEndDoc = Math.min(requestedEnd, docs.length);
     int realLen = Math.max(realEndDoc-offset,0);
     if (this.offset == offset && this.len == realLen) return this;
-    return new DocSlice(offset, realLen, docs, scores, matches, maxScore);
+    return new DocSlice(offset, realLen, docs, scores, matches, maxScore, matchesRelation);
   }
 
   @Override
@@ -139,4 +143,9 @@ public class DocSlice implements DocList, Accountable {
   public Collection<Accountable> getChildResources() {
     return Collections.emptyList();
   }
+
+  @Override
+  public TotalHits.Relation hitCountRelation() {
+    return matchesRelation;
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/search/Grouping.java b/solr/core/src/java/org/apache/solr/search/Grouping.java
index 11cbf30..c17cf58 100644
--- a/solr/core/src/java/org/apache/solr/search/Grouping.java
+++ b/solr/core/src/java/org/apache/solr/search/Grouping.java
@@ -414,7 +414,7 @@ public class Grouping {
       for (int val : idSet) {
         ids[idx++] = val;
       }
-      qr.setDocList(new DocSlice(0, sz, ids, null, maxMatches, maxScore));
+      qr.setDocList(new DocSlice(0, sz, ids, null, maxMatches, maxScore, TotalHits.Relation.EQUAL_TO));
     }
   }
 
@@ -630,7 +630,7 @@ public class Grouping {
 
       float score = groups.maxScore;
       maxScore = maxAvoidNaN(score, maxScore);
-      DocSlice docs = new DocSlice(off, Math.max(0, ids.length - off), ids, scores, groups.totalHits.value, score);
+      DocSlice docs = new DocSlice(off, Math.max(0, ids.length - off), ids, scores, groups.totalHits.value, score, TotalHits.Relation.EQUAL_TO);
 
       if (getDocList) {
         DocIterator iter = docs.iterator();
@@ -672,7 +672,7 @@ public class Grouping {
       int len = docsGathered > offset ? docsGathered - offset : 0;
       int[] docs = ArrayUtils.toPrimitive(ids.toArray(new Integer[ids.size()]));
       float[] docScores = ArrayUtils.toPrimitive(scores.toArray(new Float[scores.size()]));
-      DocSlice docSlice = new DocSlice(offset, len, docs, docScores, getMatches(), maxScore);
+      DocSlice docSlice = new DocSlice(offset, len, docs, docScores, getMatches(), maxScore, TotalHits.Relation.EQUAL_TO);
 
       if (getDocList) {
         for (int i = offset; i < docs.length; i++) {
diff --git a/solr/core/src/java/org/apache/solr/search/MaxScoreCollector.java b/solr/core/src/java/org/apache/solr/search/MaxScoreCollector.java
index abb2243..1b8987d 100644
--- a/solr/core/src/java/org/apache/solr/search/MaxScoreCollector.java
+++ b/solr/core/src/java/org/apache/solr/search/MaxScoreCollector.java
@@ -37,9 +37,7 @@ public class MaxScoreCollector extends SimpleCollector {
 
   @Override
   public ScoreMode scoreMode() {
-    // Should be TOP_SCORES but this would wrap the scorer unnecessarily since
-    // this collector is only used in a MultiCollector.
-    return ScoreMode.COMPLETE;
+    return ScoreMode.TOP_SCORES;
   }
 
   @Override
diff --git a/solr/core/src/java/org/apache/solr/search/QueryCommand.java b/solr/core/src/java/org/apache/solr/search/QueryCommand.java
index 553e022..a272878 100755
--- a/solr/core/src/java/org/apache/solr/search/QueryCommand.java
+++ b/solr/core/src/java/org/apache/solr/search/QueryCommand.java
@@ -37,6 +37,7 @@ public class QueryCommand {
   private int supersetMaxDoc;
   private int flags;
   private long timeAllowed = -1;
+  private int minExactHits = Integer.MAX_VALUE;
   private CursorMark cursorMark;
   
   public CursorMark getCursorMark() {
@@ -182,6 +183,15 @@ public class QueryCommand {
     this.timeAllowed = timeAllowed;
     return this;
   }
+
+  public int getMinExactHits() {
+    return minExactHits;
+  }
+
+  public QueryCommand setMinExactHits(int hits) {
+    this.minExactHits = hits;
+    return this;
+  }
   
   public boolean isNeedDocSet() {
     return (flags & SolrIndexSearcher.GET_DOCSET) != 0;
diff --git a/solr/core/src/java/org/apache/solr/search/QueryResultKey.java b/solr/core/src/java/org/apache/solr/search/QueryResultKey.java
index 2db1f9c..23374e8 100644
--- a/solr/core/src/java/org/apache/solr/search/QueryResultKey.java
+++ b/solr/core/src/java/org/apache/solr/search/QueryResultKey.java
@@ -16,15 +16,15 @@
  */
 package org.apache.solr.search;
 
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
-import java.util.List;
-import java.util.ArrayList;
-
 /** A hash key encapsulating a query, a list of filters, and a sort
  *
  */
@@ -37,18 +37,23 @@ public final class QueryResultKey implements Accountable {
   final SortField[] sfields;
   final List<Query> filters;
   final int nc_flags;  // non-comparable flags... ignored by hashCode and equals
+  final int minExactHits;
 
   private final int hc;  // cached hashCode
   private final long ramBytesUsed; // cached
 
   private static SortField[] defaultSort = new SortField[0];
 
-
   public QueryResultKey(Query query, List<Query> filters, Sort sort, int nc_flags) {
+    this(query, filters, sort, nc_flags, Integer.MAX_VALUE);
+  }
+
+  public QueryResultKey(Query query, List<Query> filters, Sort sort, int nc_flags, int minExactHits) {
     this.query = query;
     this.sort = sort;
     this.filters = filters;
     this.nc_flags = nc_flags;
+    this.minExactHits = minExactHits;
 
     int h = query.hashCode();
 
@@ -65,6 +70,7 @@ public final class QueryResultKey implements Accountable {
       h = h*29 + sf.hashCode();
       ramSfields += BASE_SF_RAM_BYTES_USED + RamUsageEstimator.sizeOfObject(sf.getField());
     }
+    h = h*31 + minExactHits;
 
     hc = h;
 
@@ -96,6 +102,7 @@ public final class QueryResultKey implements Accountable {
     if (this.sfields.length != other.sfields.length) return false;
     if (!this.query.equals(other.query)) return false;
     if (!unorderedCompare(this.filters, other.filters)) return false;
+    if (this.minExactHits != other.minExactHits) return false;
 
     for (int i=0; i<sfields.length; i++) {
       SortField sf1 = this.sfields[i];
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 e53cf0c..e2d8107 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -50,6 +50,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.*;
+import org.apache.lucene.search.TotalHits.Relation;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -1305,7 +1306,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
         && (flags & (NO_CHECK_QCACHE | NO_SET_QCACHE)) != ((NO_CHECK_QCACHE | NO_SET_QCACHE))) {
       // all of the current flags can be reused during warming,
       // so set all of them on the cache key.
-      key = new QueryResultKey(q, cmd.getFilterList(), cmd.getSort(), flags);
+      key = new QueryResultKey(q, cmd.getFilterList(), cmd.getSort(), flags, cmd.getMinExactHits());
       if ((flags & NO_CHECK_QCACHE) == 0) {
         superset = queryResultCache.get(key);
 
@@ -1482,7 +1483,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
    *          The Command whose properties should determine the type of TopDocsCollector to use.
    */
   private TopDocsCollector buildTopDocsCollector(int len, QueryCommand cmd) throws IOException {
-
+    int minNumFound = cmd.getMinExactHits();
     Query q = cmd.getQuery();
     if (q instanceof RankQuery) {
       RankQuery rq = (RankQuery) q;
@@ -1491,14 +1492,14 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
 
     if (null == cmd.getSort()) {
       assert null == cmd.getCursorMark() : "have cursor but no sort";
-      return TopScoreDocCollector.create(len, Integer.MAX_VALUE);
+      return TopScoreDocCollector.create(len, minNumFound);
     } else {
       // we have a sort
       final Sort weightedSort = weightSort(cmd.getSort());
       final CursorMark cursor = cmd.getCursorMark();
 
       final FieldDoc searchAfter = (null != cursor ? cursor.getSearchAfterFieldDoc() : null);
-      return TopFieldCollector.create(weightedSort, len, searchAfter, Integer.MAX_VALUE);
+      return TopFieldCollector.create(weightedSort, len, searchAfter, minNumFound);
     }
   }
 
@@ -1517,6 +1518,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
 
     ProcessedFilter pf = getProcessedFilter(cmd.getFilter(), cmd.getFilterList());
     final Query query = QueryUtils.combineQueryAndFilter(QueryUtils.makeQueryable(cmd.getQuery()), pf.filter);
+    Relation hitsRelation;
 
     // handle zero case...
     if (lastDocRequested <= 0) {
@@ -1569,8 +1571,9 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
       maxScore = totalHits > 0 ? topscore[0] : 0.0f;
       // no docs on this page, so cursor doesn't change
       qr.setNextCursorMark(cmd.getCursorMark());
+      hitsRelation = Relation.EQUAL_TO;
     } else {
-      final TopDocsCollector topCollector = buildTopDocsCollector(len, cmd);
+      final TopDocsCollector<?> topCollector = buildTopDocsCollector(len, cmd);
       MaxScoreCollector maxScoreCollector = null;
       Collector collector = topCollector;
       if ((cmd.getFlags() & GET_SCORES) != 0) {
@@ -1581,6 +1584,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
 
       totalHits = topCollector.getTotalHits();
       TopDocs topDocs = topCollector.topDocs(0, len);
+      hitsRelation = topDocs.totalHits.relation;
       if (cmd.getSort() != null && query instanceof RankQuery == false && (cmd.getFlags() & GET_SCORES) != 0) {
         TopFieldCollector.populateScores(topDocs.scoreDocs, this, query);
       }
@@ -1599,7 +1603,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
 
     int sliceLen = Math.min(lastDocRequested, nDocsReturned);
     if (sliceLen < 0) sliceLen = 0;
-    qr.setDocList(new DocSlice(0, sliceLen, ids, scores, totalHits, maxScore));
+    qr.setDocList(new DocSlice(0, sliceLen, ids, scores, totalHits, maxScore, hitsRelation));
   }
 
   // any DocSet returned is for the query only, without any filtering... that way it may
@@ -1618,6 +1622,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
 
     boolean needScores = (cmd.getFlags() & GET_SCORES) != 0;
     int maxDoc = maxDoc();
+    cmd.setMinExactHits(Integer.MAX_VALUE);// We need the full DocSet
 
     ProcessedFilter pf = getProcessedFilter(cmd.getFilter(), cmd.getFilterList());
     final Query query = QueryUtils.combineQueryAndFilter(QueryUtils.makeQueryable(cmd.getQuery()), pf.filter);
@@ -1668,7 +1673,6 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
       // no docs on this page, so cursor doesn't change
       qr.setNextCursorMark(cmd.getCursorMark());
     } else {
-
       final TopDocsCollector topCollector = buildTopDocsCollector(len, cmd);
       DocSetCollector setCollector = new DocSetCollector(maxDoc);
       MaxScoreCollector maxScoreCollector = null;
@@ -1708,7 +1712,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     int sliceLen = Math.min(lastDocRequested, nDocsReturned);
     if (sliceLen < 0) sliceLen = 0;
 
-    qr.setDocList(new DocSlice(0, sliceLen, ids, scores, totalHits, maxScore));
+    qr.setDocList(new DocSlice(0, sliceLen, ids, scores, totalHits, maxScore, TotalHits.Relation.EQUAL_TO));
     // TODO: if we collect results before the filter, we just need to intersect with
     // that filter to generate the DocSet for qr.setDocSet()
     qr.setDocSet(set);
@@ -1981,7 +1985,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     int nDocs = cmd.getSupersetMaxDoc();
     if (nDocs == 0) {
       // SOLR-2923
-      qr.getDocListAndSet().docList = new DocSlice(0, 0, new int[0], null, set.size(), 0f);
+      qr.getDocListAndSet().docList = new DocSlice(0, 0, new int[0], null, set.size(), 0f, TotalHits.Relation.EQUAL_TO);
       qr.setNextCursorMark(cmd.getCursorMark());
       return;
     }
@@ -2020,7 +2024,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     }
 
     assert topDocs.totalHits.relation == TotalHits.Relation.EQUAL_TO;
-    qr.getDocListAndSet().docList = new DocSlice(0, nDocsReturned, ids, null, topDocs.totalHits.value, 0.0f);
+    qr.getDocListAndSet().docList = new DocSlice(0, nDocsReturned, ids, null, topDocs.totalHits.value, 0.0f, topDocs.totalHits.relation);
     populateNextCursorMarkFromTopDocs(qr, cmd, topDocs);
   }
 
diff --git a/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java b/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java
index 51abc0a..2928ab2 100644
--- a/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java
+++ b/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java
@@ -1017,7 +1017,7 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
                  e.getMessage().contains(f));
     }
   }
-
+  
 //   /** this doesn't work, but if it did, this is how we'd test it. */
 //   public void testOverwriteFalse() {
 
diff --git a/solr/core/src/test/org/apache/solr/TestCrossCoreJoin.java b/solr/core/src/test/org/apache/solr/TestCrossCoreJoin.java
index 492e2c3..074e7c4 100644
--- a/solr/core/src/test/org/apache/solr/TestCrossCoreJoin.java
+++ b/solr/core/src/test/org/apache/solr/TestCrossCoreJoin.java
@@ -87,14 +87,14 @@ public class TestCrossCoreJoin extends SolrTestCaseJ4 {
   void doTestJoin(String joinPrefix) throws Exception {
     assertJQ(req("q", joinPrefix + " from=dept_id_s to=dept_s fromIndex=fromCore}cat:dev", "fl", "id",
         "debugQuery", random().nextBoolean() ? "true":"false")
-        , "/response=={'numFound':3,'start':0,'docs':[{'id':'1'},{'id':'4'},{'id':'5'}]}"
+        , "/response=={'numFound':3,'start':0,'numFoundExact':true,'docs':[{'id':'1'},{'id':'4'},{'id':'5'}]}"
     );
 
     // find people that develop stuff - but limit via filter query to a name of "john"
     // this tests filters being pushed down to queries (SOLR-3062)
     assertJQ(req("q", joinPrefix + " from=dept_id_s to=dept_s fromIndex=fromCore}cat:dev", "fl", "id", "fq", "name:john",
         "debugQuery", random().nextBoolean() ? "true":"false")
-        , "/response=={'numFound':1,'start':0,'docs':[{'id':'1'}]}"
+        , "/response=={'numFound':1,'start':0,'numFoundExact':true,'docs':[{'id':'1'}]}"
     );
   }
 
diff --git a/solr/core/src/test/org/apache/solr/TestDistributedSearch.java b/solr/core/src/test/org/apache/solr/TestDistributedSearch.java
index f634f3e..fb62107 100644
--- a/solr/core/src/test/org/apache/solr/TestDistributedSearch.java
+++ b/solr/core/src/test/org/apache/solr/TestDistributedSearch.java
@@ -209,7 +209,10 @@ public class TestDistributedSearch extends BaseDistributedSearchTestCase {
     query("q","*:*", "sort",i1+" desc", "fl","*,score");
     query("q","*:*", "sort","n_tl1 asc", "fl","*,score"); 
     query("q","*:*", "sort","n_tl1 desc");
+    
     handle.put("maxScore", SKIPVAL);
+    testMinExactHits();
+    
     query("q","{!func}"+i1);// does not expect maxScore. So if it comes ,ignore it. JavaBinCodec.writeSolrDocumentList()
     //is agnostic of request params.
     handle.remove("maxScore");
@@ -1082,11 +1085,38 @@ public class TestDistributedSearch extends BaseDistributedSearchTestCase {
     assertEquals(new EnumFieldValue(11, "Critical"),
                  rsp.getFieldStatsInfo().get(fieldName).getMax());
 
-    handle.put("severity", UNORDERED); // this is stupid, but stats.facet doesn't garuntee order
+    handle.put("severity", UNORDERED); // this is stupid, but stats.facet doesn't guarantee order
     query("q", "*:*", "stats", "true", "stats.field", fieldName, 
           "stats.facet", fieldName);
   }
 
+  private void testMinExactHits() throws Exception {
+    assertIsExactHitCount("q","{!cache=false}dog OR men OR cow OR country OR dumpty", CommonParams.MIN_EXACT_HITS, "200", CommonParams.ROWS, "2", CommonParams.SORT, "score desc, id asc");
+    assertIsExactHitCount("q","{!cache=false}dog OR men OR cow OR country OR dumpty", CommonParams.MIN_EXACT_HITS, "-1", CommonParams.ROWS, "2", CommonParams.SORT, "score desc, id asc");
+    assertIsExactHitCount("q","{!cache=false}dog OR men OR cow OR country OR dumpty", CommonParams.MIN_EXACT_HITS, "1", CommonParams.ROWS, "200", CommonParams.SORT, "score desc, id asc");
+    assertIsExactHitCount("q","{!cache=false}dog OR men OR cow OR country OR dumpty", "facet", "true", "facet.field", s1, CommonParams.MIN_EXACT_HITS,"1", CommonParams.ROWS, "200", CommonParams.SORT, "score desc, id asc");
+    assertIsExactHitCount("q","{!cache=false}id:1", CommonParams.MIN_EXACT_HITS,"1", CommonParams.ROWS, "1");
+    assertApproximatedHitCount("q","{!cache=false}dog OR men OR cow OR country OR dumpty", CommonParams.MIN_EXACT_HITS,"2", CommonParams.ROWS, "2", CommonParams.SORT, "score desc, id asc");
+  }
+  
+  private void assertIsExactHitCount(Object... requestParams) throws Exception {
+    QueryResponse response = query(requestParams);
+    assertNotNull("Expecting exact hit count in response: " + response.getResults().toString(),
+        response.getResults().getNumFoundExact());
+    assertTrue("Expecting exact hit count in response: " + response.getResults().toString(),
+        response.getResults().getNumFoundExact());
+  }
+  
+  private void assertApproximatedHitCount(Object...requestParams) throws Exception {
+    handle.put("numFound", SKIPVAL);
+    QueryResponse response = query(requestParams);
+    assertNotNull("Expecting numFoundExact in response: " + response.getResults().toString(),
+        response.getResults().getNumFoundExact());
+    assertFalse("Expecting aproximated results in response: " + response.getResults().toString(),
+        response.getResults().getNumFoundExact());
+    handle.remove("numFound", SKIPVAL);
+  }
+
   /** comparing results with facet.method=uif */
   private void queryAndCompareUIF(Object ... params) throws Exception {
     final QueryResponse expect = query(params);
diff --git a/solr/core/src/test/org/apache/solr/TestGroupingSearch.java b/solr/core/src/test/org/apache/solr/TestGroupingSearch.java
index 45c75d7..8bec510 100644
--- a/solr/core/src/test/org/apache/solr/TestGroupingSearch.java
+++ b/solr/core/src/test/org/apache/solr/TestGroupingSearch.java
@@ -16,20 +16,6 @@
  */
 package org.apache.solr;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-
 import org.apache.solr.client.solrj.impl.BinaryResponseParser;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
@@ -50,6 +36,20 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
 public class TestGroupingSearch extends SolrTestCaseJ4 {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -245,11 +245,11 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
 
     assertJQ(
         req("q", "*:*", "start", "1", "group", "true", "group.field", "id", "group.main", "true"),
-        "/response=={'numFound':3,'start':1,'docs':[{'id':'2'},{'id':'3'}]}"
+        "/response=={'numFound':3,'start':1,'numFoundExact':true,'docs':[{'id':'2'},{'id':'3'}]}"
     );
     assertJQ(
         req("q", "*:*", "start", "1", "rows", "1", "group", "true", "group.field", "id", "group.main", "true"),
-        "/response=={'numFound':3,'start':1,'docs':[{'id':'2'}]}"
+        "/response=={'numFound':3,'start':1,'numFoundExact':true,'docs':[{'id':'2'}]}"
     );
   }
 
@@ -264,7 +264,7 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
 
     assertJQ(
         req("q", "*:*", "start", "2", "rows", "1", "group", "true", "group.field", "id", "group.main", "true"),
-        "/response=={'numFound':5,'start':2,'docs':[{'id':'3'}]}"
+        "/response=={'numFound':5,'start':2,'numFoundExact':true,'docs':[{'id':'3'}]}"
     );
   }
 
@@ -323,12 +323,12 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
 
     assertJQ(
         req("q", "*:*", "sort", "sum(value1_i, value2_i) desc", "rows", "1", "group", "true", "group.field", "id", "fl", "id"),
-        "/grouped=={'id':{'matches':5,'groups':[{'groupValue':'5','doclist':{'numFound':1,'start':0,'docs':[{'id':'5'}]}}]}}"
+        "/grouped=={'id':{'matches':5,'groups':[{'groupValue':'5','doclist':{'numFound':1,'start':0,numFoundExact:true,'docs':[{'id':'5'}]}}]}}"
     );
 
     assertJQ(
         req("q", "*:*", "sort", "geodist(45.18014,-93.87742,store) asc", "rows", "1", "group", "true", "group.field", "id", "fl", "id"),
-        "/grouped=={'id':{'matches':5,'groups':[{'groupValue':'1','doclist':{'numFound':1,'start':0,'docs':[{'id':'1'}]}}]}}"
+        "/grouped=={'id':{'matches':5,'groups':[{'groupValue':'1','doclist':{'numFound':1,'start':0,numFoundExact:true,'docs':[{'id':'1'}]}}]}}"
     );
   }
 
@@ -347,7 +347,7 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
         "value1_s1", "fl", "id", "facet", "true", "facet.field", "value3_s1", "group.truncate", "false");
     assertJQ(
         req,
-        "/grouped=={'value1_s1':{'matches':5,'groups':[{'groupValue':'1','doclist':{'numFound':3,'start':0,'docs':[{'id':'1'}]}}]}}",
+        "/grouped=={'value1_s1':{'matches':5,'groups':[{'groupValue':'1','doclist':{'numFound':3,'start':0,numFoundExact:true,'docs':[{'id':'1'}]}}]}}",
         "/facet_counts=={'facet_queries':{},'facet_fields':{'value3_s1':['a',3,'b',2]}," + EMPTY_FACETS + "}"
     );
 
@@ -356,7 +356,7 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
         "value1_s1", "fl", "id", "facet", "true", "facet.field", "value3_s1", "group.truncate", "true");
     assertJQ(
         req,
-        "/grouped=={'value1_s1':{'matches':5,'groups':[{'groupValue':'1','doclist':{'numFound':3,'start':0,'docs':[{'id':'1'}]}}]}}",
+        "/grouped=={'value1_s1':{'matches':5,'groups':[{'groupValue':'1','doclist':{'numFound':3,'start':0,numFoundExact:true,'docs':[{'id':'1'}]}}]}}",
         "/facet_counts=={'facet_queries':{},'facet_fields':{'value3_s1':['a',1,'b',1]}," + EMPTY_FACETS + "}"
     );
 
@@ -365,7 +365,7 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
         "strdist(1,value1_s1,edit)", "fl", "id", "facet", "true", "facet.field", "value3_s1", "group.truncate", "true");
     assertJQ(
         req,
-        "/grouped=={'strdist(1,value1_s1,edit)':{'matches':5,'groups':[{'groupValue':1.0,'doclist':{'numFound':3,'start':0,'docs':[{'id':'1'}]}}]}}",
+        "/grouped=={'strdist(1,value1_s1,edit)':{'matches':5,'groups':[{'groupValue':1.0,'doclist':{'numFound':3,'start':0,numFoundExact:true,'docs':[{'id':'1'}]}}]}}",
         "/facet_counts=={'facet_queries':{},'facet_fields':{'value3_s1':['a',1,'b',1]}," + EMPTY_FACETS + "}"
     );
 
@@ -374,7 +374,7 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
         "facet.field", "value3_s1", "group.truncate", "true");
     assertJQ(
         req,
-        "/grouped=={'value4_i':{'matches':5,'groups':[{'groupValue':1,'doclist':{'numFound':3,'start':0,'docs':[{'id':'1'}]}}]}}",
+        "/grouped=={'value4_i':{'matches':5,'groups':[{'groupValue':1,'doclist':{'numFound':3,'start':0,numFoundExact:true,'docs':[{'id':'1'}]}}]}}",
         "/facet_counts=={'facet_queries':{},'facet_fields':{'value3_s1':['a',1,'b',1]}," + EMPTY_FACETS + "}"
     );
 
@@ -383,7 +383,7 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
         "facet.field", "{!ex=v}value3_s1", "group.truncate", "true", "fq", "{!tag=v}value3_s1:b");
     assertJQ(
         req,
-        "/grouped=={'value4_i':{'matches':2,'groups':[{'groupValue':2,'doclist':{'numFound':2,'start':0,'docs':[{'id':'3'}]}}]}}",
+        "/grouped=={'value4_i':{'matches':2,'groups':[{'groupValue':2,'doclist':{'numFound':2,'start':0,numFoundExact:true,'docs':[{'id':'3'}]}}]}}",
         "/facet_counts=={'facet_queries':{},'facet_fields':{'value3_s1':['a',1,'b',1]}," + EMPTY_FACETS + "}"
     );
 
@@ -392,7 +392,7 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
         "facet.field", "{!ex=v}value3_s1", "group.truncate", "false", "fq", "{!tag=v}value3_s1:b");
     assertJQ(
         req,
-        "/grouped=={'value4_i':{'matches':2,'groups':[{'groupValue':2,'doclist':{'numFound':2,'start':0,'docs':[{'id':'3'}]}}]}}",
+        "/grouped=={'value4_i':{'matches':2,'groups':[{'groupValue':2,'doclist':{'numFound':2,'start':0,numFoundExact:true,'docs':[{'id':'3'}]}}]}}",
         "/facet_counts=={'facet_queries':{},'facet_fields':{'value3_s1':['a',3,'b',2]}," + EMPTY_FACETS + "}"
     );
 
@@ -401,7 +401,7 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
         "facet.field", "{!ex=v}value3_s1", "group.truncate", "true", "fq", "{!tag=v}value3_s1:b");
     assertJQ(
         req,
-        "/grouped=={'sub(value4_i,1)':{'matches':2,'groups':[{'groupValue':1.0,'doclist':{'numFound':2,'start':0,'docs':[{'id':'3'}]}}]}}",
+        "/grouped=={'sub(value4_i,1)':{'matches':2,'groups':[{'groupValue':1.0,'doclist':{'numFound':2,'start':0,numFoundExact:true,'docs':[{'id':'3'}]}}]}}",
         "/facet_counts=={'facet_queries':{},'facet_fields':{'value3_s1':['a',1,'b',1]}," + EMPTY_FACETS + "}"
     );
   }
@@ -424,7 +424,7 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
         "facet.query", "{!ex=chk key=LM3}bday:[2012-10-18T00:00:00Z TO 2013-01-17T23:59:59Z]");
     assertJQ(
         req,
-        "/grouped=={'cat_sI':{'matches':2,'groups':[{'groupValue':'a','doclist':{'numFound':1,'start':0,'docs':[{'id':'5'}]}}]}}",
+        "/grouped=={'cat_sI':{'matches':2,'groups':[{'groupValue':'a','doclist':{'numFound':1,'start':0,numFoundExact:true,'docs':[{'id':'5'}]}}]}}",
         "/facet_counts=={'facet_queries':{'LW1':2,'LM1':2,'LM3':2},'facet_fields':{}," + EMPTY_FACETS + "}"
     );
   }
@@ -463,33 +463,33 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
       ,"/responseHeader=={'_SKIP_':'QTime', 'status':0}"   // partial match by skipping some elements
       ,"/responseHeader=={'_MATCH_':'status', 'status':0}" // partial match by only including some elements
       ,"/grouped=={'"+f+"':{'matches':10,'groups':[\n" +
-              "{'groupValue':1,'doclist':{'numFound':3,'start':0,'docs':[{'id':'8'}]}}," +
-              "{'groupValue':3,'doclist':{'numFound':2,'start':0,'docs':[{'id':'3'}]}}," +
-              "{'groupValue':2,'doclist':{'numFound':3,'start':0,'docs':[{'id':'4'}]}}," +
-              "{'groupValue':5,'doclist':{'numFound':1,'start':0,'docs':[{'id':'1'}]}}," +
-              "{'groupValue':4,'doclist':{'numFound':1,'start':0,'docs':[{'id':'2'}]}}" +
+              "{'groupValue':1,'doclist':{'numFound':3,'start':0,numFoundExact:true,'docs':[{'id':'8'}]}}," +
+              "{'groupValue':3,'doclist':{'numFound':2,'start':0,numFoundExact:true,'docs':[{'id':'3'}]}}," +
+              "{'groupValue':2,'doclist':{'numFound':3,'start':0,numFoundExact:true,'docs':[{'id':'4'}]}}," +
+              "{'groupValue':5,'doclist':{'numFound':1,'start':0,numFoundExact:true,'docs':[{'id':'1'}]}}," +
+              "{'groupValue':4,'doclist':{'numFound':1,'start':0,numFoundExact:true,'docs':[{'id':'2'}]}}" +
             "]}}"
     );
 
     // test that filtering cuts down the result set
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.field",f, "fl","id", "fq",f+":2")
       ,"/grouped=={'"+f+"':{'matches':3,'groups':[" +
-            "{'groupValue':2,'doclist':{'numFound':3,'start':0,'docs':[{'id':'4'}]}}" +
+            "{'groupValue':2,'doclist':{'numFound':3,'start':0,numFoundExact:true,'docs':[{'id':'4'}]}}" +
             "]}}"
     );
 
     // test limiting the number of groups returned
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.field",f, "fl","id", "rows","2")
       ,"/grouped=={'"+f+"':{'matches':10,'groups':[" +
-              "{'groupValue':1,'doclist':{'numFound':3,'start':0,'docs':[{'id':'8'}]}}," +
-              "{'groupValue':3,'doclist':{'numFound':2,'start':0,'docs':[{'id':'3'}]}}" +
+              "{'groupValue':1,'doclist':{'numFound':3,'start':0,numFoundExact:true,'docs':[{'id':'8'}]}}," +
+              "{'groupValue':3,'doclist':{'numFound':2,'start':0,numFoundExact:true,'docs':[{'id':'3'}]}}" +
             "]}}"
     );
 
     // test offset into group list
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.field",f, "fl","id", "rows","1", "start","1")
       ,"/grouped=={'"+f+"':{'matches':10,'groups':[" +
-              "{'groupValue':3,'doclist':{'numFound':2,'start':0,'docs':[{'id':'3'}]}}" +
+              "{'groupValue':3,'doclist':{'numFound':2,'start':0,numFoundExact:true,'docs':[{'id':'3'}]}}" +
             "]}}"
     );
 
@@ -502,24 +502,24 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
     // test increasing the docs per group returned
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.field",f, "fl","id", "rows","2", "group.limit","3")
       ,"/grouped=={'"+f+"':{'matches':10,'groups':[" +
-            "{'groupValue':1,'doclist':{'numFound':3,'start':0,'docs':[{'id':'8'},{'id':'10'},{'id':'5'}]}}," +
-            "{'groupValue':3,'doclist':{'numFound':2,'start':0,'docs':[{'id':'3'},{'id':'6'}]}}" +
+            "{'groupValue':1,'doclist':{'numFound':3,'start':0,numFoundExact:true,'docs':[{'id':'8'},{'id':'10'},{'id':'5'}]}}," +
+            "{'groupValue':3,'doclist':{'numFound':2,'start':0,numFoundExact:true,'docs':[{'id':'3'},{'id':'6'}]}}" +
           "]}}"
     );
 
     // test offset into each group
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.field",f, "fl","id", "rows","2", "group.limit","3", "group.offset","1")
       ,"/grouped=={'"+f+"':{'matches':10,'groups':[" +
-            "{'groupValue':1,'doclist':{'numFound':3,'start':1,'docs':[{'id':'10'},{'id':'5'}]}}," +
-            "{'groupValue':3,'doclist':{'numFound':2,'start':1,'docs':[{'id':'6'}]}}" +
+            "{'groupValue':1,'doclist':{'numFound':3,'start':1,'numFoundExact':true,'docs':[{'id':'10'},{'id':'5'}]}}," +
+            "{'groupValue':3,'doclist':{'numFound':2,'start':1,'numFoundExact':true,'docs':[{'id':'6'}]}}" +
           "]}}"
     );
 
     // test big offset into each group
      assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.field",f, "fl","id", "rows","2", "group.limit","3", "group.offset","10")
       ,"/grouped=={'"+f+"':{'matches':10,'groups':[" +
-            "{'groupValue':1,'doclist':{'numFound':3,'start':10,'docs':[]}}," +
-            "{'groupValue':3,'doclist':{'numFound':2,'start':10,'docs':[]}}" +
+            "{'groupValue':1,'doclist':{'numFound':3,'start':10,'numFoundExact':true,'docs':[]}}," +
+            "{'groupValue':3,'doclist':{'numFound':2,'start':10,'numFoundExact':true,'docs':[]}}" +
           "]}}"
     );
 
@@ -527,8 +527,8 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.field",f, "fl","id,score", "rows","2", "group.limit","2", "indent","off")
       ,"/grouped/"+f+"/groups==" +
             "[" +
-              "{'groupValue':1,'doclist':{'numFound':3,'start':0,'maxScore':10.0,'docs':[{'id':'8','score':10.0},{'id':'10','score':3.0}]}}," +
-              "{'groupValue':3,'doclist':{'numFound':2,'start':0,'maxScore':7.0,'docs':[{'id':'3','score':7.0},{'id':'6','score':2.0}]}}" +
+              "{'groupValue':1,'doclist':{'numFound':3,'start':0,numFoundExact:true,'maxScore':10.0,'docs':[{'id':'8','score':10.0},{'id':'10','score':3.0}]}}," +
+              "{'groupValue':3,'doclist':{'numFound':2,'start':0,numFoundExact:true,'maxScore':7.0,'docs':[{'id':'3','score':7.0},{'id':'6','score':2.0}]}}" +
             "]"
 
     );
@@ -537,8 +537,8 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
     String func = "add("+f+","+f+")";
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.func", func  , "fl","id", "rows","2")
       ,"/grouped=={'"+func+"':{'matches':10,'groups':[" +
-              "{'groupValue':2.0,'doclist':{'numFound':3,'start':0,'docs':[{'id':'8'}]}}," +
-              "{'groupValue':6.0,'doclist':{'numFound':2,'start':0,'docs':[{'id':'3'}]}}" +
+              "{'groupValue':2.0,'doclist':{'numFound':3,'start':0,numFoundExact:true,'docs':[{'id':'8'}]}}," +
+              "{'groupValue':6.0,'doclist':{'numFound':2,'start':0,numFoundExact:true,'docs':[{'id':'3'}]}}" +
             "]}}"
     );
 
@@ -560,7 +560,7 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
      ///////////////////////// group.query
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.query","id:[2 TO 5]", "fl","id", "group.limit","3")
        ,"/grouped=={'id:[2 TO 5]':{'matches':10," +
-           "'doclist':{'numFound':4,'start':0,'docs':[{'id':'3'},{'id':'4'},{'id':'2'}]}}}"
+           "'doclist':{'numFound':4,'start':0,numFoundExact:true,'docs':[{'id':'3'},{'id':'4'},{'id':'2'}]}}}"
     );
 
     // group.query that matches nothing
@@ -571,50 +571,50 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
                  "group.query","id:1000", 
                  "fl","id", 
                  "group.limit","3")
-             ,"/grouped/id:[2 TO 5]=={'matches':10,'doclist':{'numFound':4,'start':0,'docs':[{'id':'3'},{'id':'4'},{'id':'2'}]}}"
-             ,"/grouped/id:1000=={'matches':10,'doclist':{'numFound':0,'start':0,'docs':[]}}"
+             ,"/grouped/id:[2 TO 5]=={'matches':10,'doclist':{'numFound':4,'start':0,numFoundExact:true,'docs':[{'id':'3'},{'id':'4'},{'id':'2'}]}}"
+             ,"/grouped/id:1000=={'matches':10,'doclist':{'numFound':0,'start':0,numFoundExact:true,'docs':[]}}"
     );
 
     // group.query and sort
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.query",f+":1", "fl","id,score", "rows","2", "group.limit","2", "sort",f+" desc, score desc", "indent","off")
         ,"/grouped/"+f+":1==" +
-            "{'matches':10,'doclist':{'numFound':3,'start':0,'maxScore':10.0,'docs':[{'id':'8','score':10.0},{'id':'10','score':3.0}]}},"
+            "{'matches':10,'doclist':{'numFound':3,'start':0,numFoundExact:true,'maxScore':10.0,'docs':[{'id':'8','score':10.0},{'id':'10','score':3.0}]}},"
     );
     // group.query with fl=score and default sort
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.query",f+":1", "fl","id,score", "rows","2", "group.limit","2", "sort", "score desc", "indent","off")
         ,"/grouped/"+f+":1==" +
-            "{'matches':10,'doclist':{'numFound':3,'start':0,'maxScore':10.0,'docs':[{'id':'8','score':10.0},{'id':'10','score':3.0}]}},"
+            "{'matches':10,'doclist':{'numFound':3,'start':0,numFoundExact:true,'maxScore':10.0,'docs':[{'id':'8','score':10.0},{'id':'10','score':3.0}]}},"
     );
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.query",f+":1", "fl","id", "rows","2", "group.limit","2", "indent","off")
         ,"/grouped/"+f+":1==" +
-            "{'matches':10,'doclist':{'numFound':3,'start':0,'docs':[{'id':'8'},{'id':'10'}]}},"
+            "{'matches':10,'doclist':{'numFound':3,'start':0,numFoundExact:true,'docs':[{'id':'8'},{'id':'10'}]}},"
     );
 
     // group.query and offset
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.query","id:[2 TO 5]", "fl","id", "group.limit","3", "group.offset","2")
        ,"/grouped=={'id:[2 TO 5]':{'matches':10," +
-           "'doclist':{'numFound':4,'start':2,'docs':[{'id':'2'},{'id':'5'}]}}}"
+           "'doclist':{'numFound':4,'start':2,'numFoundExact':true,'docs':[{'id':'2'},{'id':'5'}]}}}"
     );
 
     // group.query and big offset
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.query","id:[2 TO 5]", "fl","id", "group.limit","3", "group.offset","10")
        ,"/grouped=={'id:[2 TO 5]':{'matches':10," +
-           "'doclist':{'numFound':4,'start':10,'docs':[]}}}"
+           "'doclist':{'numFound':4,'start':10,'numFoundExact':true,'docs':[]}}}"
     );
 
     ///////////////////////// group.query as main result
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.query","id:[2 TO 5]", "fl","id", "rows","3", "group.main","true")
-       ,"/response=={'numFound':4,'start':0,'docs':[{'id':'3'},{'id':'4'},{'id':'2'}]}"
+       ,"/response=={'numFound':4,'start':0,numFoundExact:true,'docs':[{'id':'3'},{'id':'4'},{'id':'2'}]}"
     );
 
     // group.query and offset
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.query","id:[2 TO 5]", "fl","id", "rows","3", "start","2", "group.main","true")
-       ,"/response=={'numFound':4,'start':2,'docs':[{'id':'2'},{'id':'5'}]}"
+       ,"/response=={'numFound':4,'start':2,'numFoundExact':true,'docs':[{'id':'2'},{'id':'5'}]}"
     );
 
     // group.query and big offset
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.query","id:[2 TO 5]", "fl","id", "rows","3", "start","10", "group.main","true")
-       ,"/response=={'numFound':4,'start':10,'docs':[]}"
+       ,"/response=={'numFound':4,'start':10,'numFoundExact':true,'docs':[]}"
     );
 
 
@@ -625,46 +625,46 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
         "group.field",f,
         "rows","1",
         "fl","id", "group.limit","2")
-       ,"/grouped/id:[2 TO 5]=={'matches':10,'doclist':{'numFound':4,'start':0,'docs':[{'id':'3'},{'id':'4'}]}}"
-       ,"/grouped/id:[5 TO 5]=={'matches':10,'doclist':{'numFound':1,'start':0,'docs':[{'id':'5'}]}}"        
-       ,"/grouped/"+f+"=={'matches':10,'groups':[{'groupValue':1,'doclist':{'numFound':3,'start':0,'docs':[{'id':'8'},{'id':'10'}]}}]}"
+       ,"/grouped/id:[2 TO 5]=={'matches':10,'doclist':{'numFound':4,'start':0,numFoundExact:true,'docs':[{'id':'3'},{'id':'4'}]}}"
+       ,"/grouped/id:[5 TO 5]=={'matches':10,'doclist':{'numFound':1,'start':0,numFoundExact:true,'docs':[{'id':'5'}]}}"        
+       ,"/grouped/"+f+"=={'matches':10,'groups':[{'groupValue':1,'doclist':{'numFound':3,'start':0,numFoundExact:true,'docs':[{'id':'8'},{'id':'10'}]}}]}"
     );
 
 
     ///////////////////////// group.field as main result
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.field",f, "fl","id", "group.main","true")
-        ,"/response=={'numFound':10,'start':0,'docs':[{'id':'8'},{'id':'3'},{'id':'4'},{'id':'1'},{'id':'2'}]}"
+        ,"/response=={'numFound':10,'start':0,numFoundExact:true,'docs':[{'id':'8'},{'id':'3'},{'id':'4'},{'id':'1'},{'id':'2'}]}"
     );
     // test that rows limits #docs
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.field",f, "fl","id", "rows","3", "group.main","true")
-        ,"/response=={'numFound':10,'start':0,'docs':[{'id':'8'},{'id':'3'},{'id':'4'}]}"
+        ,"/response=={'numFound':10,'start':0,numFoundExact:true,'docs':[{'id':'8'},{'id':'3'},{'id':'4'}]}"
     );
     // small  offset
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.field",f, "fl","id", "rows","2", "start","1", "group.main","true")
-        ,"/response=={'numFound':10,'start':1,'docs':[{'id':'3'},{'id':'4'}]}"
+        ,"/response=={'numFound':10,'start':1,'numFoundExact':true,'docs':[{'id':'3'},{'id':'4'}]}"
     );
     // large offset
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.field",f, "fl","id", "rows","2", "start","20", "group.main","true")
-        ,"/response=={'numFound':10,'start':20,'docs':[]}"
+        ,"/response=={'numFound':10,'start':20,'numFoundExact':true,'docs':[]}"
     );
     // group.limit>1
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.field",f, "fl","id", "rows","3", "group.limit","2", "group.main","true")
-        ,"/response=={'numFound':10,'start':0,'docs':[{'id':'8'},{'id':'10'},{'id':'3'}]}"
+        ,"/response=={'numFound':10,'start':0,numFoundExact:true,'docs':[{'id':'8'},{'id':'10'},{'id':'3'}]}"
     );
     // group.limit>1 with start>0
     assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.field",f, "fl","id", "rows","3", "start","1", "group.limit","2", "group.main","true")
-        ,"/response=={'numFound':10,'start':1,'docs':[{'id':'10'},{'id':'3'},{'id':'6'}]}"
+        ,"/response=={'numFound':10,'start':1,'numFoundExact':true,'docs':[{'id':'10'},{'id':'3'},{'id':'6'}]}"
     );
 
     ///////////////////////// group.format == simple
     assertJQ(req("fq", filt, "q", "{!func}" + f2, "group", "true", "group.field", f, "fl", "id", "rows", "3", "start", "1", "group.limit", "2", "group.format", "simple")
         , "/grouped/foo_i=={'matches':10,'doclist':"
-        + "{'numFound':10,'start':1,'docs':[{'id':'10'},{'id':'3'},{'id':'6'}]}}"
+        + "{'numFound':10,'start':1,'numFoundExact':true,'docs':[{'id':'10'},{'id':'3'},{'id':'6'}]}}"
     );
 
     //////////////////////// grouping where main query matches nothing
     assertJQ(req("fq", filt, "q", "bogus_s:nothing", "group", "true", "group.field", f, "fl", "id", "group.limit", "2", "group.format", "simple")
-        , "/grouped/foo_i=={'matches':0,'doclist':{'numFound':0,'start':0,'docs':[]}}"
+        , "/grouped/foo_i=={'matches':0,'doclist':{'numFound':0,'start':0,numFoundExact:true,'docs':[]}}"
     );
     assertJQ(req("fq",filt,  "q","bogus_s:nothing", "group","true",
         "group.query","id:[2 TO 5]",
@@ -672,8 +672,8 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
         "group.field",f,
         "rows","1",
         "fl","id", "group.limit","2")
-       ,"/grouped/id:[2 TO 5]=={'matches':0,'doclist':{'numFound':0,'start':0,'docs':[]}}"
-       ,"/grouped/id:[5 TO 5]=={'matches':0,'doclist':{'numFound':0,'start':0,'docs':[]}}"        
+       ,"/grouped/id:[2 TO 5]=={'matches':0,'doclist':{'numFound':0,'start':0,numFoundExact:true,'docs':[]}}"
+       ,"/grouped/id:[5 TO 5]=={'matches':0,'doclist':{'numFound':0,'start':0,numFoundExact:true,'docs':[]}}"        
        ,"/grouped/"+f+"=={'matches':0,'groups':[]}"
     );
     assertJQ(req("fq",filt,  
@@ -683,8 +683,8 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
                  "group.query","id:1000", 
                  "fl","id", 
                  "group.limit","3")
-             ,"/grouped/id:[2 TO 5]=={'matches':0,'doclist':{'numFound':0,'start':0,'docs':[]}}"
-             ,"/grouped/id:1000=={'matches':0,'doclist':{'numFound':0,'start':0,'docs':[]}}"
+             ,"/grouped/id:[2 TO 5]=={'matches':0,'doclist':{'numFound':0,'start':0,numFoundExact:true,'docs':[]}}"
+             ,"/grouped/id:1000=={'matches':0,'doclist':{'numFound':0,'start':0,numFoundExact:true,'docs':[]}}"
     );
 
   }
@@ -709,7 +709,7 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
         "facet.query", "{!ex=chk key=LM3}bday:[2012-10-18T00:00:00Z TO 2013-01-17T23:59:59Z]");
     assertJQ(
         req,
-        "/grouped=={'"+FOO_STRING_DOCVAL_FIELD+"':{'matches':2,'groups':[{'groupValue':'a','doclist':{'numFound':1,'start':0,'docs':[{'id':'5'}]}}]}}",
+        "/grouped=={'"+FOO_STRING_DOCVAL_FIELD+"':{'matches':2,'groups':[{'groupValue':'a','doclist':{'numFound':1,'start':0,numFoundExact:true,'docs':[{'id':'5'}]}}]}}",
         "/facet_counts=={'facet_queries':{'LW1':2,'LM1':2,'LM3':2},'facet_fields':{}," + EMPTY_FACETS + "}"
     );
   }
@@ -730,10 +730,10 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
 
     assertJQ(req(params, "group.field", "date_dt", "sort", "id asc"),
         "/grouped=={'date_dt':{'matches':5,'ngroups':4, 'groups':" +
-            "[{'groupValue':'2012-11-20T00:00:00Z','doclist':{'numFound':2,'start':0,'docs':[{'id':'1'},{'id':'3'}]}}," +
-            "{'groupValue':'2012-11-21T00:00:00Z','doclist':{'numFound':1,'start':0,'docs':[{'id':'2'}]}}," +
-            "{'groupValue':'2013-01-15T00:00:00Z','doclist':{'numFound':1,'start':0,'docs':[{'id':'4'}]}}," +
-            "{'groupValue':null,'doclist':{'numFound':1,'start':0,'docs':[{'id':'5'}]}}" +
+            "[{'groupValue':'2012-11-20T00:00:00Z','doclist':{'numFound':2,'start':0,numFoundExact:true,'docs':[{'id':'1'},{'id':'3'}]}}," +
+            "{'groupValue':'2012-11-21T00:00:00Z','doclist':{'numFound':1,'start':0,numFoundExact:true,'docs':[{'id':'2'}]}}," +
+            "{'groupValue':'2013-01-15T00:00:00Z','doclist':{'numFound':1,'start':0,numFoundExact:true,'docs':[{'id':'4'}]}}," +
+            "{'groupValue':null,'doclist':{'numFound':1,'start':0,numFoundExact:true,'docs':[{'id':'5'}]}}" +
             "]}}"
     );
   }
@@ -940,6 +940,31 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
     } // end index iter
 
   }
+  
+  @Test
+  public void testGroupWithMinExactHitCount() throws Exception {
+    final int NUM_DOCS = 20;
+    for (int i = 0; i < NUM_DOCS ; i++) {
+      assertU(adoc("id", String.valueOf(i), FOO_STRING_FIELD, "Book1"));
+      assertU(commit());
+    }
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("q", FOO_STRING_FIELD + ":Book1");
+    assertQ(req(params, CommonParams.MIN_EXACT_HITS, "2", CommonParams.ROWS, "2")
+        ,"/response/result[@numFoundExact='false']"
+    );
+    params.set("group", true);
+    params.set("group.field", FOO_STRING_FIELD);
+    assertQ(req(params)
+        ,"/response/lst[@name='grouped']/lst[@name='"+FOO_STRING_FIELD+"']/arr[@name='groups']/lst[1]/result[@numFoundExact='true']"
+    );
+    
+    assertQ(req(params, CommonParams.MIN_EXACT_HITS, "2", CommonParams.ROWS, "2")
+        ,"/response/lst[@name='grouped']/lst[@name='"+FOO_STRING_FIELD+"']/arr[@name='groups']/lst[1]/result[@numFoundExact='true']"
+    );
+    
+    
+  }
 
   public static Object buildGroupedResult(IndexSchema schema, List<Grp> sortedGroups, int start, int rows, int group_offset, int group_limit, boolean includeNGroups) {
     Map<String,Object> result = new LinkedHashMap<>();
@@ -952,7 +977,7 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
     if (includeNGroups) {
       result.put("ngroups", sortedGroups.size());
     }
-    List groupList = new ArrayList();
+    List<Map<String,Object>> groupList = new ArrayList<>();
     result.put("groups", groupList);
 
     for (int i=start; i<sortedGroups.size(); i++) {
@@ -967,7 +992,8 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
       group.put("doclist", resultSet);
       resultSet.put("numFound", grp.docs.size());
       resultSet.put("start", group_offset);
-      List docs = new ArrayList();
+      resultSet.put("numFoundExact", true);
+      List<Map<String,Object>> docs = new ArrayList<>();
       resultSet.put("docs", docs);
       for (int j=group_offset; j<grp.docs.size(); j++) {
         if (group_limit != -1 && docs.size() >= group_limit) break;
diff --git a/solr/core/src/test/org/apache/solr/TestJoin.java b/solr/core/src/test/org/apache/solr/TestJoin.java
index 6a46b7e..f3d9b07 100644
--- a/solr/core/src/test/org/apache/solr/TestJoin.java
+++ b/solr/core/src/test/org/apache/solr/TestJoin.java
@@ -80,37 +80,37 @@ public class TestJoin extends SolrTestCaseJ4 {
     ModifiableSolrParams p = params("sort","id asc");
 
     assertJQ(req(p, "q", buildJoinRequest(DEPT_FIELD, DEPT_ID_FIELD, "title:MTS"), "fl","id")
-        ,"/response=={'numFound':3,'start':0,'docs':[{'id':'10'},{'id':'12'},{'id':'13'}]}"
+        ,"/response=={'numFound':3,'start':0,'numFoundExact':true,'docs':[{'id':'10'},{'id':'12'},{'id':'13'}]}"
     );
 
     // empty from
     assertJQ(req(p, "q", buildJoinRequest("noexist_ss_dv", DEPT_ID_FIELD, "*:*", "fl","id"))
-        ,"/response=={'numFound':0,'start':0,'docs':[]}"
+        ,"/response=={'numFound':0,'start':0,'numFoundExact':true,'docs':[]}"
     );
 
     // empty to
     assertJQ(req(p, "q", buildJoinRequest(DEPT_FIELD, "noexist_ss_dv", "*:*"), "fl","id")
-        ,"/response=={'numFound':0,'start':0,'docs':[]}"
+        ,"/response=={'numFound':0,'start':0,'numFoundExact':true,'docs':[]}"
     );
 
     // self join... return everyone in same dept(s) as Dave
     assertJQ(req(p, "q", buildJoinRequest(DEPT_FIELD, DEPT_FIELD, "name:dave"), "fl","id")
-        ,"/response=={'numFound':3,'start':0,'docs':[{'id':'1'},{'id':'4'},{'id':'5'}]}"
+        ,"/response=={'numFound':3,'start':0,'numFoundExact':true,'docs':[{'id':'1'},{'id':'4'},{'id':'5'}]}"
     );
 
     // from single-value to multi-value
     assertJQ(req(p, "q", buildJoinRequest(DEPT_ID_FIELD, DEPT_FIELD, "text:develop"), "fl","id")
-        ,"/response=={'numFound':3,'start':0,'docs':[{'id':'1'},{'id':'4'},{'id':'5'}]}"
+        ,"/response=={'numFound':3,'start':0,'numFoundExact':true,'docs':[{'id':'1'},{'id':'4'},{'id':'5'}]}"
     );
 
     // from multi-value to single-value
     assertJQ(req(p, "q",buildJoinRequest(DEPT_FIELD, DEPT_ID_FIELD, "title:MTS"), "fl","id", "debugQuery","true")
-        ,"/response=={'numFound':3,'start':0,'docs':[{'id':'10'},{'id':'12'},{'id':'13'}]}"
+        ,"/response=={'numFound':3,'start':0,'numFoundExact':true,'docs':[{'id':'10'},{'id':'12'},{'id':'13'}]}"
     );
 
     // expected outcome for a sub query matching dave joined against departments
     final String davesDepartments =
-        "/response=={'numFound':2,'start':0,'docs':[{'id':'10'},{'id':'13'}]}";
+        "/response=={'numFound':2,'start':0,'numFoundExact':true,'docs':[{'id':'10'},{'id':'13'}]}";
 
     // straight forward query
     assertJQ(req(p, "q", buildJoinRequest(DEPT_FIELD, DEPT_ID_FIELD, "name:dave"), "fl","id"),
@@ -134,7 +134,7 @@ public class TestJoin extends SolrTestCaseJ4 {
     // find people that develop stuff - but limit via filter query to a name of "john"
     // this tests filters being pushed down to queries (SOLR-3062)
     assertJQ(req(p, "q", buildJoinRequest(DEPT_ID_FIELD, DEPT_FIELD, "text:develop"), "fl","id", "fq", "name:john")
-        ,"/response=={'numFound':1,'start':0,'docs':[{'id':'1'}]}"
+        ,"/response=={'numFound':1,'start':0,'numFoundExact':true,'docs':[{'id':'1'}]}"
     );
   }
 
@@ -171,7 +171,7 @@ public class TestJoin extends SolrTestCaseJ4 {
 
     // non-DV/text field.
     assertJQ(req(p, "q","{!join from=title to=title}name:dave", "fl","id")
-        ,"/response=={'numFound':2,'start':0,'docs':[{'id':'3'},{'id':'4'}]}"
+        ,"/response=={'numFound':2,'start':0,'numFoundExact':true,'docs':[{'id':'3'},{'id':'4'}]}"
     );
   }
 
@@ -248,6 +248,7 @@ public class TestJoin extends SolrTestCaseJ4 {
         Map<String,Object> resultSet = new LinkedHashMap<>();
         resultSet.put("numFound", docList.size());
         resultSet.put("start", 0);
+        resultSet.put("numFoundExact", true);
         resultSet.put("docs", sortedDocs);
 
         // todo: use different join queries for better coverage
diff --git a/solr/core/src/test/org/apache/solr/core/QueryResultKeyTest.java b/solr/core/src/test/org/apache/solr/core/QueryResultKeyTest.java
index dc1e48b..775ae5e 100644
--- a/solr/core/src/test/org/apache/solr/core/QueryResultKeyTest.java
+++ b/solr/core/src/test/org/apache/solr/core/QueryResultKeyTest.java
@@ -136,7 +136,21 @@ public class QueryResultKeyTest extends SolrTestCaseJ4 {
     }
     assert minIters <= iter;
   }
-
+  
+  public void testMinExactHits() {
+    int[] nums = smallArrayOfRandomNumbers();
+    final Query base = new FlatHashTermQuery("base");
+    assertKeyEquals(new QueryResultKey(base, buildFiltersFromNumbers(nums), null, 0, 10),
+        new QueryResultKey(base, buildFiltersFromNumbers(nums), null, 0, 10));
+    assertKeyNotEquals(new QueryResultKey(base, buildFiltersFromNumbers(nums), null, 0, 10),
+        new QueryResultKey(base, buildFiltersFromNumbers(nums), null, 0, 20));
+    assertKeyNotEquals(new QueryResultKey(base, buildFiltersFromNumbers(nums), null, 0, 10),
+        new QueryResultKey(base, buildFiltersFromNumbers(nums), null, 0));//Integer.MAX_VALUE
+    assertKeyEquals(new QueryResultKey(base, buildFiltersFromNumbers(nums), null, 0, Integer.MAX_VALUE),
+        new QueryResultKey(base, buildFiltersFromNumbers(nums), null, 0));
+    
+  }
+  
   /**
    * does bi-directional equality check as well as verifying hashCode
    */
diff --git a/solr/core/src/test/org/apache/solr/handler/tagger/TaggerTest.java b/solr/core/src/test/org/apache/solr/handler/tagger/TaggerTest.java
index 9bc0bcc..71d5af5 100644
--- a/solr/core/src/test/org/apache/solr/handler/tagger/TaggerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/tagger/TaggerTest.java
@@ -80,7 +80,7 @@ public class TaggerTest extends TaggerTestCase {
         "    </arr>\n" +
         "  </lst>\n" +
         "</arr>\n" +
-        "<result name=\"response\" numFound=\"1\" start=\"0\">\n" +
+        "<result name=\"response\" numFound=\"1\" start=\"0\" numFoundExact=\"true\">\n" +
         "  <doc>\n" +
         "    <str name=\"id\">1</str>\n" +
         "    <str name=\"name\">London Business School</str>\n" +
@@ -109,7 +109,7 @@ public class TaggerTest extends TaggerTestCase {
         "    </arr>\n" +
         "  </lst>\n" +
         "</arr>\n" +
-        "<result name=\"response\" numFound=\"1\" start=\"0\">\n" +
+        "<result name=\"response\" numFound=\"1\" start=\"0\" numFoundExact=\"true\">\n" +
         "  <doc>\n" +
         "    <str name=\"id\">1</str>\n" +
         "    <str name=\"name\">London Business School</str>\n" +
@@ -307,7 +307,7 @@ public class TaggerTest extends TaggerTestCase {
         "\n" +
         "<int name=\"tagsCount\">0</int>\n" +
         "<arr name=\"tags\"/>\n" +
-        "<result name=\"response\" numFound=\"0\" start=\"0\">\n" +
+        "<result name=\"response\" numFound=\"0\" start=\"0\" numFoundExact=\"true\">\n" +
         "</result>\n" +
         "</response>\n";
     assertEquals(expected, rspStr);
diff --git a/solr/core/src/test/org/apache/solr/request/TestFaceting.java b/solr/core/src/test/org/apache/solr/request/TestFaceting.java
index 59b3a41..81d4d71 100644
--- a/solr/core/src/test/org/apache/solr/request/TestFaceting.java
+++ b/solr/core/src/test/org/apache/solr/request/TestFaceting.java
@@ -27,7 +27,9 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BytesRef;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.FacetParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.uninverting.DocTermOrds;
 import org.junit.After;
 import org.junit.BeforeClass;
@@ -931,5 +933,28 @@ public class TestFaceting extends SolrTestCaseJ4 {
         "//lst[@name='facet_fields']/lst[@name='title_ws']/int[2][@name='Book2']",
         "//lst[@name='facet_fields']/lst[@name='title_ws']/int[3][@name='Book3']");
   }
+  
+  @Test
+  public void testFacetCountsWithMinExactHits() throws Exception {
+    final int NUM_DOCS = 20;
+    for (int i = 0; i < NUM_DOCS ; i++) {
+      assertU(adoc("id", String.valueOf(i), "title_ws", "Book1"));
+      assertU(commit());
+    }
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("q", "title_ws:Book1");
+    params.set(FacetParams.FACET, "true");
+    params.set(FacetParams.FACET_FIELD, "title_ws");
+    assertQ(req(params),
+        "//lst[@name='facet_fields']/lst[@name='title_ws']/int[1][@name='Book1'][.='20']"
+        ,"//*[@numFoundExact='true']"
+        ,"//*[@numFound='" + NUM_DOCS + "']");
+    
+    // It doesn't matter if we request minExactHits, when requesting facets, the numFound value is precise
+    assertQ(req(params, CommonParams.MIN_EXACT_HITS, "2", CommonParams.ROWS, "2"),
+        "//lst[@name='facet_fields']/lst[@name='title_ws']/int[1][@name='Book1'][.='20']"
+        ,"//*[@numFoundExact='true']"
+        ,"//*[@numFound='" + NUM_DOCS + "']");
+  }
 }
 
diff --git a/solr/core/src/test/org/apache/solr/response/JSONWriterTest.java b/solr/core/src/test/org/apache/solr/response/JSONWriterTest.java
index 8a6cb71..484b487 100644
--- a/solr/core/src/test/org/apache/solr/response/JSONWriterTest.java
+++ b/solr/core/src/test/org/apache/solr/response/JSONWriterTest.java
@@ -180,8 +180,8 @@ public class JSONWriterTest extends SolrTestCaseJ4 {
                result.contains("\"id\"") &&
                result.contains("\"score\"") && result.contains("_children_"));
 
-    String expectedResult = "{'response':{'numFound':1,'start':0,'maxScore':0.7,'docs':[{'id':'1', 'score':'0.7'," +
-        " '_children_':{'numFound':1,'start':0,'docs':[{'id':'2', 'score':'0.4', 'path':['a>b', 'a>b>c']}] }}] }}";
+    String expectedResult = "{'response':{'numFound':1,'start':0,'maxScore':0.7, 'numFoundExact':true,'docs':[{'id':'1', 'score':'0.7'," +
+        " '_children_':{'numFound':1,'start':0,'numFoundExact':true,'docs':[{'id':'2', 'score':'0.4', 'path':['a>b', 'a>b>c']}] }}] }}";
     String error = JSONTestUtil.match(result, "=="+expectedResult);
     assertNull("response validation failed with error: " + error, error);
 
diff --git a/solr/core/src/test/org/apache/solr/response/TestPHPSerializedResponseWriter.java b/solr/core/src/test/org/apache/solr/response/TestPHPSerializedResponseWriter.java
index 6418825..e383743 100644
--- a/solr/core/src/test/org/apache/solr/response/TestPHPSerializedResponseWriter.java
+++ b/solr/core/src/test/org/apache/solr/response/TestPHPSerializedResponseWriter.java
@@ -94,7 +94,7 @@ public class TestPHPSerializedResponseWriter extends SolrTestCaseJ4 {
     rsp.addResponse(sdl);
     
     w.write(buf, req, rsp);
-    assertEquals("a:1:{s:8:\"response\";a:3:{s:8:\"numFound\";i:0;s:5:\"start\";i:0;s:4:\"docs\";a:2:{i:0;a:6:{s:2:\"id\";s:1:\"1\";s:5:\"data1\";s:5:\"hello\";s:5:\"data2\";i:42;s:5:\"data3\";b:1;s:5:\"data4\";a:2:{s:7:\"data4.1\";s:7:\"hashmap\";s:7:\"data4.2\";s:5:\"hello\";}s:5:\"data5\";a:3:{i:0;s:7:\"data5.1\";i:1;s:7:\"data5.2\";i:2;s:7:\"data5.3\";}}i:1;a:1:{s:2:\"id\";s:1:\"2\";}}}}", 
+    assertEquals("a:1:{s:8:\"response\";a:4:{s:8:\"numFound\";i:0;s:5:\"start\";i:0;s:13:\"numFoundExact\";b:1;s:4:\"docs\";a:2:{i:0;a:6:{s:2:\"id\";s:1:\"1\";s:5:\"data1\";s:5:\"hello\";s:5:\"data2\";i:42;s:5:\"data3\";b:1;s:5:\"data4\";a:2:{s:7:\"data4.1\";s:7:\"hashmap\";s:7:\"data4.2\";s:5:\"hello\";}s:5:\"data5\";a:3:{i:0;s:7:\"data5.1\";i:1;s:7:\"data5.2\";i:2;s:7:\"data5.3\";}}i:1;a:1:{s:2:\"id\";s:1:\"2\";}}}}", 
                  buf.toString());
     req.close();
   }
diff --git a/solr/core/src/test/org/apache/solr/search/SolrIndexSearcherTest.java b/solr/core/src/test/org/apache/solr/search/SolrIndexSearcherTest.java
new file mode 100644
index 0000000..309af8a
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/search/SolrIndexSearcherTest.java
@@ -0,0 +1,200 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TotalHits;
+import org.apache.solr.SolrTestCaseJ4;
+import org.junit.Before;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+public class SolrIndexSearcherTest extends SolrTestCaseJ4 {
+  
+  private final static int NUM_DOCS = 20;
+
+  @BeforeClass
+  public static void setUpClass() throws Exception {
+    initCore("solrconfig.xml", "schema.xml");
+    for (int i = 0 ; i < NUM_DOCS ; i ++) {
+      assertU(adoc("id", String.valueOf(i), "field1_s", "foo", "field2_s", String.valueOf(i % 2), "field3_s", String.valueOf(i)));
+      assertU(commit());
+    }
+  }
+  
+  @Before
+  public void setUp() throws Exception {
+    assertU(adoc("id", "1", "field1_s", "foo", "field2_s", "1", "field3_s", "1"));
+    assertU(commit());
+    super.setUp();
+  }
+  
+  public void testMinExactHitsLongValue() {
+    assertQ("test query on empty index",
+        req("q", "field1_s:foo", 
+            "minExactHits", Long.toString(10L * Integer.MAX_VALUE),
+            "rows", "2")
+        ,"//*[@numFoundExact='true']"
+        ,"//*[@numFound='" + NUM_DOCS + "']"
+        );
+  }
+  
+  public void testMinExactHits() {
+    assertQ("minExactHits is lower than numFound,should produce approximated results",
+            req("q", "field1_s:foo", 
+                "minExactHits", "2",
+                "rows", "2")
+            ,"//*[@numFoundExact='false']"
+            ,"//*[@numFound<='" + NUM_DOCS + "']"
+            );
+    assertQ("minExactHits is higher than numFound,should produce exact results",
+        req("q", "field1_s:foo", 
+            "minExactHits", "200",
+            "rows", "2")
+        ,"//*[@numFoundExact='true']"
+        ,"//*[@numFound='" + NUM_DOCS + "']"
+        );
+  }
+  
+  private void assertMatchesEqual(int expectedCount, QueryResult qr) {
+    assertEquals(expectedCount, qr.getDocList().matches());
+    assertEquals(TotalHits.Relation.EQUAL_TO, qr.getDocList().hitCountRelation());
+  }
+  
+  private void assertMatchesGraterThan(int expectedCount, QueryResult qr) {
+    assertTrue("Expecting returned matches to be greater than " + expectedCount + " but got " + qr.getDocList().matches(),
+        expectedCount >= qr.getDocList().matches());
+    assertEquals(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, qr.getDocList().hitCountRelation());
+  }
+  
+  public void testLowMinExactHitsGeneratesApproximation() throws IOException {
+    h.getCore().withSearcher(searcher -> {
+      QueryCommand cmd = new QueryCommand();
+      cmd.setMinExactHits(NUM_DOCS / 2);
+      cmd.setQuery(new TermQuery(new Term("field1_s", "foo")));
+      QueryResult qr = new QueryResult();
+      searcher.search(qr, cmd);
+      assertMatchesGraterThan(NUM_DOCS, qr);
+      return null;
+    });
+    
+    h.getCore().withSearcher(searcher -> {
+      QueryCommand cmd = new QueryCommand();
+      cmd.setMinExactHits(1);
+      cmd.setLen(1);
+      // We need to disable cache, otherwise the search will be done for 20 docs (cache window size) which brings up the minExactHits
+      cmd.setFlags(SolrIndexSearcher.NO_CHECK_QCACHE | SolrIndexSearcher.NO_SET_QCACHE);
+      cmd.setQuery(new TermQuery(new Term("field2_s", "1")));
+      QueryResult qr = new QueryResult();
+      searcher.search(qr, cmd);
+      assertMatchesGraterThan(NUM_DOCS/2, qr);
+      return null;
+    });
+  }
+  
+  public void testHighMinExactHitsGeneratesExactCount() throws IOException {
+    h.getCore().withSearcher(searcher -> {
+      QueryCommand cmd = new QueryCommand();
+      cmd.setMinExactHits(NUM_DOCS);
+      cmd.setQuery(new TermQuery(new Term("field1_s", "foo")));
+      QueryResult qr = new QueryResult();
+      searcher.search(qr, cmd);
+      assertMatchesEqual(NUM_DOCS, qr);
+      return null;
+    });
+    
+    h.getCore().withSearcher(searcher -> {
+      QueryCommand cmd = new QueryCommand();
+      cmd.setMinExactHits(NUM_DOCS);
+      cmd.setQuery(new TermQuery(new Term("field2_s", "1")));
+      QueryResult qr = new QueryResult();
+      searcher.search(qr, cmd);
+      assertMatchesEqual(NUM_DOCS/2, qr);
+      return null;
+    });
+  }
+  
+  public void testLowMinExactHitsWithQueryResultCache() throws IOException {
+    h.getCore().withSearcher(searcher -> {
+      QueryCommand cmd = new QueryCommand();
+      cmd.setMinExactHits(NUM_DOCS / 2);
+      cmd.setQuery(new TermQuery(new Term("field1_s", "foo")));
+      searcher.search(new QueryResult(), cmd);
+      QueryResult qr = new QueryResult();
+      searcher.search(qr, cmd);
+      assertMatchesGraterThan(NUM_DOCS, qr);
+      return null;
+    });
+  }
+  
+  public void testHighMinExactHitsWithQueryResultCache() throws IOException {
+    h.getCore().withSearcher(searcher -> {
+      QueryCommand cmd = new QueryCommand();
+      cmd.setMinExactHits(NUM_DOCS);
+      cmd.setQuery(new TermQuery(new Term("field1_s", "foo")));
+      searcher.search(new QueryResult(), cmd);
+      QueryResult qr = new QueryResult();
+      searcher.search(qr, cmd);
+      assertMatchesEqual(NUM_DOCS, qr);
+      return null;
+    });
+  }
+  
+  public void testMinExactHitsMoreRows() throws IOException {
+    h.getCore().withSearcher(searcher -> {
+      QueryCommand cmd = new QueryCommand();
+      cmd.setMinExactHits(2);
+      cmd.setLen(NUM_DOCS);
+      cmd.setQuery(new TermQuery(new Term("field1_s", "foo")));
+      QueryResult qr = new QueryResult();
+      searcher.search(qr, cmd);
+      assertMatchesEqual(NUM_DOCS, qr);
+      return null;
+    });
+  }
+  
+  public void testMinExactHitsMatchWithDocSet() throws IOException {
+    h.getCore().withSearcher(searcher -> {
+      QueryCommand cmd = new QueryCommand();
+      cmd.setNeedDocSet(true);
+      cmd.setMinExactHits(2);
+      cmd.setQuery(new TermQuery(new Term("field1_s", "foo")));
+      searcher.search(new QueryResult(), cmd);
+      QueryResult qr = new QueryResult();
+      searcher.search(qr, cmd);
+      assertMatchesEqual(NUM_DOCS, qr);
+      return null;
+    });
+  }
+  
+  public void testMinExactHitsWithMaxScoreRequested() throws IOException {
+    h.getCore().withSearcher(searcher -> {
+      QueryCommand cmd = new QueryCommand();
+      cmd.setMinExactHits(2);
+      cmd.setFlags(SolrIndexSearcher.GET_SCORES);
+      cmd.setQuery(new TermQuery(new Term("field1_s", "foo")));
+      searcher.search(new QueryResult(), cmd);
+      QueryResult qr = new QueryResult();
+      searcher.search(qr, cmd);
+      assertMatchesGraterThan(NUM_DOCS, qr);
+      assertNotEquals(Float.NaN, qr.getDocList().maxScore());
+      return null;
+    });
+  }
+}
diff --git a/solr/core/src/test/org/apache/solr/search/TestAddFieldRealTimeGet.java b/solr/core/src/test/org/apache/solr/search/TestAddFieldRealTimeGet.java
index 34f7cde..aaa5440 100644
--- a/solr/core/src/test/org/apache/solr/search/TestAddFieldRealTimeGet.java
+++ b/solr/core/src/test/org/apache/solr/search/TestAddFieldRealTimeGet.java
@@ -59,11 +59,13 @@ public class TestAddFieldRealTimeGet extends TestRTGBase {
     String newFieldType = "string";
     String newFieldValue = "xyz";
 
+    ignoreException("unknown field");
     assertFailedU("Should fail due to unknown field '" + newFieldName + "'", 
                   adoc("id", "1", newFieldName, newFieldValue));
+    unIgnoreException("unknown field");
 
     IndexSchema schema = h.getCore().getLatestSchema();
-    SchemaField newField = schema.newField(newFieldName, newFieldType, Collections.<String,Object>emptyMap());
+    SchemaField newField = schema.newField(newFieldName, newFieldType, Collections.emptyMap());
     IndexSchema newSchema = schema.addField(newField);
     h.getCore().setLatestSchema(newSchema);
     
@@ -74,7 +76,7 @@ public class TestAddFieldRealTimeGet extends TestRTGBase {
     assertJQ(req("qt","/get", "id","1", "fl","id,"+newFieldName),
              "=={'doc':{'id':'1'," + newFieldKeyValue + "}}");
     assertJQ(req("qt","/get","ids","1", "fl","id,"+newFieldName),
-             "=={'response':{'numFound':1,'start':0,'docs':[{'id':'1'," + newFieldKeyValue + "}]}}");
+             "=={'response':{'numFound':1,'start':0,'numFoundExact':true,'docs':[{'id':'1'," + newFieldKeyValue + "}]}}");
 
     assertU(commit());
 
@@ -83,6 +85,6 @@ public class TestAddFieldRealTimeGet extends TestRTGBase {
     assertJQ(req("qt","/get", "id","1", "fl","id,"+newFieldName),
         "=={'doc':{'id':'1'," + newFieldKeyValue + "}}");
     assertJQ(req("qt","/get","ids","1", "fl","id,"+newFieldName),
-        "=={'response':{'numFound':1,'start':0,'docs':[{'id':'1'," + newFieldKeyValue + "}]}}");
+        "=={'response':{'numFound':1,'start':0,'numFoundExact':true,'docs':[{'id':'1'," + newFieldKeyValue + "}]}}");
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/search/TestDocSet.java b/solr/core/src/test/org/apache/solr/search/TestDocSet.java
index cd18b5e..00ee6ec 100644
--- a/solr/core/src/test/org/apache/solr/search/TestDocSet.java
+++ b/solr/core/src/test/org/apache/solr/search/TestDocSet.java
@@ -39,6 +39,7 @@ import org.apache.lucene.index.StoredFieldVisitor;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TotalHits;
 import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.FixedBitSet;
@@ -114,7 +115,7 @@ public class TestDocSet extends SolrTestCase {
       arr[i] = iter.nextDoc();
     }
 
-    return new DocSlice(offset, len, arr, null, len*2, 100.0f);
+    return new DocSlice(offset, len, arr, null, len*2, 100.0f, TotalHits.Relation.EQUAL_TO);
   }
 
 
diff --git a/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java b/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java
index 40aea9f..8020fc4 100644
--- a/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java
+++ b/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java
@@ -71,7 +71,7 @@ public class TestRealTimeGet extends TestRTGBase {
     );
     assertJQ(req("qt","/get","ids","1", "fl","id")
         ,"=={" +
-        "  'response':{'numFound':1,'start':0,'docs':[" +
+        "  'response':{'numFound':1,'start':0,'numFoundExact':true,'docs':[" +
         "      {" +
         "        'id':'1'}]" +
         "  }}}"
@@ -98,7 +98,7 @@ public class TestRealTimeGet extends TestRTGBase {
     );
     assertJQ(req("qt","/get","ids","1", "fl","id")
         ,"=={" +
-        "  'response':{'numFound':1,'start':0,'docs':[" +
+        "  'response':{'numFound':1,'start':0,'numFoundExact':true,'docs':[" +
         "      {" +
         "        'id':'1'}]" +
         "  }}}"
@@ -113,7 +113,7 @@ public class TestRealTimeGet extends TestRTGBase {
         ,"=={'doc':null}"
     );
     assertJQ(req("qt","/get","ids","1")
-        ,"=={'response':{'numFound':0,'start':0,'docs':[]}}"
+        ,"=={'response':{'numFound':0,'start':0,'numFoundExact':true,'docs':[]}}"
     );
 
 
diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
index aea04b3..c3d31ff 100644
--- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
+++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
@@ -3131,7 +3131,7 @@ public class TestJsonFacets extends SolrTestCaseHS {
             "  }" +
             "}" )
 
-        , "response=={numFound:10,start:0,docs:[]}"
+        , "response=={numFound:10,start:0,numFoundExact:true,docs:[]}"
         , "facets=={ count:10," +
             "types:{" +
             "    buckets:[ {val:page, count:10, in_books:2, via_field:2, via_query:2 } ]}" +
diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetsWithNestedObjects.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetsWithNestedObjects.java
index 6e05491..1e5b203 100644
--- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetsWithNestedObjects.java
+++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetsWithNestedObjects.java
@@ -130,7 +130,7 @@ public class TestJsonFacetsWithNestedObjects extends SolrTestCaseHS{
             "  }" +
             "}"
         )
-        , "response=={numFound:2,start:0,docs:[" +
+        , "response=={numFound:2,start:0,'numFoundExact':true,docs:[" +
             "      {id:book1_c1," +
             "        comment_t:\"A great start to what looks like an epic series!\"}," +
             "      {id:book2_c1," +
@@ -167,7 +167,7 @@ public class TestJsonFacetsWithNestedObjects extends SolrTestCaseHS{
             "  }" +
             "}"
         )
-        , "response=={numFound:2,start:0,docs:[" +
+        , "response=={numFound:2,start:0,'numFoundExact':true,docs:[" +
             "      {id:book1," +
             "        title_t:\"The Way of Kings\"}," +
             "      {id:book2," +
@@ -213,7 +213,7 @@ public class TestJsonFacetsWithNestedObjects extends SolrTestCaseHS{
             "        facet: {" +
             "           in_books: \"unique(_root_)\" }}}}}" )
 
-        , "response=={numFound:2,start:0,docs:[" +
+        , "response=={numFound:2,start:0,'numFoundExact':true,docs:[" +
             "      {id:book1," +
             "        title_t:\"The Way of Kings\"}," +
             "      {id:book2," +
@@ -267,7 +267,7 @@ public class TestJsonFacetsWithNestedObjects extends SolrTestCaseHS{
             "        facet: {" +
             "           in_books: \"unique(_root_)\" }}}}}" )
 
-        , "response=={numFound:2,start:0,docs:[" +
+        , "response=={numFound:2,start:0,'numFoundExact':true,docs:[" +
             "      {id:book1," +
             "        title_t:\"The Way of Kings\"}," +
             "      {id:book2," +
@@ -324,7 +324,7 @@ public class TestJsonFacetsWithNestedObjects extends SolrTestCaseHS{
             "           in_books: \"unique(_root_)\" }}"+
         "}" )
 
-        , "response=={numFound:0,start:0,docs:[]}"
+        , "response=={numFound:0,start:0,'numFoundExact':true,docs:[]}"
         , "facets=={ count:0," +
             "comments_for_author:{" +
             "    buckets:[ {val:mary,    count:1, in_books:1} ]}," +
@@ -364,7 +364,7 @@ public class TestJsonFacetsWithNestedObjects extends SolrTestCaseHS{
             "  }" +
             "}" )
 
-        , "response=={numFound:2,start:0,docs:[]}"
+        , "response=={numFound:2,start:0,'numFoundExact':true,docs:[]}"
         , "facets=={ count:2," +
             "types:{" +
             "    buckets:[ {val:review, count:5, in_books1:2, in_books2:2, "
diff --git a/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPNoScore.java b/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPNoScore.java
index 491c2b3..59c481b 100644
--- a/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPNoScore.java
+++ b/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPNoScore.java
@@ -78,41 +78,41 @@ public class TestScoreJoinQPNoScore extends SolrTestCaseJ4 {
   //  );
 
     assertJQ(req("q","{!join from=dept_ss to=dept_id_s"+whateverScore()+"}title_s:MTS", "fl","id")
-        ,"/response=={'numFound':3,'start':0,'docs':[{'id':'10'},{'id':'12'},{'id':'13'}]}"
+        ,"/response=={'numFound':3,'start':0,'numFoundExact':true,'docs':[{'id':'10'},{'id':'12'},{'id':'13'}]}"
     );
 
     // empty from
     assertJQ(req("q","{!join from=noexist_s to=dept_id_s"+whateverScore()+"}*:*", "fl","id")
-        ,"/response=={'numFound':0,'start':0,'docs':[]}"
+        ,"/response=={'numFound':0,'start':0,'numFoundExact':true,'docs':[]}"
     );
 
     // empty to
     assertJQ(req("q","{!join from=dept_ss to=noexist_s"+whateverScore()+"}*:*", "fl","id")
-        ,"/response=={'numFound':0,'start':0,'docs':[]}"
+        ,"/response=={'numFound':0,'start':0,'numFoundExact':true,'docs':[]}"
     );
 
     // self join... return everyone with she same title as Dave
     assertJQ(req("q","{!join from=title_s to=title_s"+whateverScore()+"}name_s:dave", "fl","id")
-        ,"/response=={'numFound':2,'start':0,'docs':[{'id':'3'},{'id':'4'}]}"
+        ,"/response=={'numFound':2,'start':0,'numFoundExact':true,'docs':[{'id':'3'},{'id':'4'}]}"
     );
 
     // find people that develop stuff
     assertJQ(req("q","{!join from=dept_id_s to=dept_ss"+whateverScore()+"}text_t:develop", "fl","id")
-        ,"/response=={'numFound':3,'start':0,'docs':[{'id':'1'},{'id':'4'},{'id':'5'}]}"
+        ,"/response=={'numFound':3,'start':0,'numFoundExact':true,'docs':[{'id':'1'},{'id':'4'},{'id':'5'}]}"
     );
 
     // self join on multivalued text_t field
     assertJQ(req("q","{!join from=title_s to=title_s"+whateverScore()+"}name_s:dave", "fl","id")
-        ,"/response=={'numFound':2,'start':0,'docs':[{'id':'3'},{'id':'4'}]}"
+        ,"/response=={'numFound':2,'start':0,'numFoundExact':true,'docs':[{'id':'3'},{'id':'4'}]}"
     );
 
     assertJQ(req("q","{!join from=dept_ss to=dept_id_s"+whateverScore()+"}title_s:MTS", "fl","id", "debugQuery","true")
-        ,"/response=={'numFound':3,'start':0,'docs':[{'id':'10'},{'id':'12'},{'id':'13'}]}"
+        ,"/response=={'numFound':3,'start':0,'numFoundExact':true,'docs':[{'id':'10'},{'id':'12'},{'id':'13'}]}"
     );
     
     // expected outcome for a sub query matching dave joined against departments
     final String davesDepartments = 
-      "/response=={'numFound':2,'start':0,'docs':[{'id':'10'},{'id':'13'}]}";
+      "/response=={'numFound':2,'start':0,'numFoundExact':true,'docs':[{'id':'10'},{'id':'13'}]}";
 
     // straight forward query
     assertJQ(req("q","{!join from=dept_ss to=dept_id_s"+whateverScore()+"}name_s:dave", 
@@ -144,17 +144,17 @@ public class TestScoreJoinQPNoScore extends SolrTestCaseJ4 {
     // find people that develop stuff - but limit via filter query to a name of "john"
     // this tests filters being pushed down to queries (SOLR-3062)
     assertJQ(req("q","{!join from=dept_id_s to=dept_ss"+whateverScore()+"}text_t:develop", "fl","id", "fq", "name_s:john")
-             ,"/response=={'numFound':1,'start':0,'docs':[{'id':'1'}]}"
+             ,"/response=={'numFound':1,'start':0,'numFoundExact':true,'docs':[{'id':'1'}]}"
             );
     
 
    assertJQ(req("q","{!join from=dept_ss to=dept_id_s"+whateverScore()+"}title_s:MTS", "fl","id"
           )
-          ,"/response=={'numFound':3,'start':0,'docs':[{'id':'10'},{'id':'12'},{'id':'13'}]}");
+          ,"/response=={'numFound':3,'start':0,'numFoundExact':true,'docs':[{'id':'10'},{'id':'12'},{'id':'13'}]}");
 
       // find people that develop stuff, even if it's requested as single value
     assertJQ(req("q","{!join from=dept_id_s to=dept_ss"+whateverScore()+"}text_t:develop", "fl","id")
-        ,"/response=={'numFound':3,'start':0,'docs':[{'id':'1'},{'id':'4'},{'id':'5'}]}"
+        ,"/response=={'numFound':3,'start':0,'numFoundExact':true,'docs':[{'id':'1'},{'id':'4'},{'id':'5'}]}"
     );
 
   }
@@ -278,6 +278,7 @@ public class TestScoreJoinQPNoScore extends SolrTestCaseJ4 {
         Map<String,Object> resultSet = new LinkedHashMap<String,Object>();
         resultSet.put("numFound", docList.size());
         resultSet.put("start", 0);
+        resultSet.put("numFoundExact", true);
         resultSet.put("docs", sortedDocs);
 
         // todo: use different join queries for better coverage
diff --git a/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPScore.java b/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPScore.java
index 0dc0ccf..8813b64 100644
--- a/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPScore.java
+++ b/solr/core/src/test/org/apache/solr/search/join/TestScoreJoinQPScore.java
@@ -88,7 +88,7 @@ public class TestScoreJoinQPScore extends SolrTestCaseJ4 {
 
     // Search for product
     assertJQ(req("q", "{!join from=" + idField + " to=" + toField + " score=None}name:name2", "fl", "id")
-        , "/response=={'numFound':2,'start':0,'docs':[{'id':'5'},{'id':'6'}]}");
+        , "/response=={'numFound':2,'start':0,'numFoundExact':true,'docs':[{'id':'5'},{'id':'6'}]}");
     
     /*Query joinQuery =
         JoinUtil.createJoinQuery(idField, false, toField, new TermQuery(new Term("name", "name2")), indexSearcher, ScoreMode.None);
@@ -99,7 +99,7 @@ public class TestScoreJoinQPScore extends SolrTestCaseJ4 {
     assertEquals(5, result.scoreDocs[1].doc);
     */
     assertJQ(req("q", "{!join from=" + idField + " to=" + toField + " score=None}name:name1", "fl", "id")
-        , "/response=={'numFound':2,'start':0,'docs':[{'id':'2'},{'id':'3'}]}");
+        , "/response=={'numFound':2,'start':0,'numFoundExact':true,'docs':[{'id':'2'},{'id':'3'}]}");
 
     /*joinQuery = JoinUtil.createJoinQuery(idField, false, toField, new TermQuery(new Term("name", "name1")), indexSearcher, ScoreMode.None);
     result = indexSearcher.search(joinQuery, 10);
@@ -109,7 +109,7 @@ public class TestScoreJoinQPScore extends SolrTestCaseJ4 {
 
     // Search for offer
     assertJQ(req("q", "{!join from=" + toField + " to=" + idField + " score=None}id:5", "fl", "id")
-        , "/response=={'numFound':1,'start':0,'docs':[{'id':'4'}]}");
+        , "/response=={'numFound':1,'start':0,'numFoundExact':true,'docs':[{'id':'4'}]}");
     /*joinQuery = JoinUtil.createJoinQuery(toField, false, idField, new TermQuery(new Term("id", "5")), indexSearcher, ScoreMode.None);
     result = indexSearcher.search(joinQuery, 10);
     assertEquals(1, result.totalHits);
@@ -122,10 +122,10 @@ public class TestScoreJoinQPScore extends SolrTestCaseJ4 {
   public void testDeleteByScoreJoinQuery() throws Exception {
     indexDataForScorring();
     String joinQuery = "{!join from=" + toField + " to=" + idField + " score=Max}title:random";
-    assertJQ(req("q", joinQuery, "fl", "id"), "/response=={'numFound':2,'start':0,'docs':[{'id':'1'},{'id':'4'}]}");
+    assertJQ(req("q", joinQuery, "fl", "id"), "/response=={'numFound':2,'start':0,'numFoundExact':true,'docs':[{'id':'1'},{'id':'4'}]}");
     assertU(delQ(joinQuery));
     assertU(commit());
-    assertJQ(req("q", joinQuery, "fl", "id"), "/response=={'numFound':0,'start':0,'docs':[]}");
+    assertJQ(req("q", joinQuery, "fl", "id"), "/response=={'numFound':0,'start':0,'numFoundExact':true,'docs':[]}");
   }
 
   public void testSimpleWithScoring() throws Exception {
@@ -133,7 +133,7 @@ public class TestScoreJoinQPScore extends SolrTestCaseJ4 {
 
     // Search for movie via subtitle
     assertJQ(req("q", "{!join from=" + toField + " to=" + idField + " score=Max}title:random", "fl", "id")
-        , "/response=={'numFound':2,'start':0,'docs':[{'id':'1'},{'id':'4'}]}");
+        , "/response=={'numFound':2,'start':0,'numFoundExact':true,'docs':[{'id':'1'},{'id':'4'}]}");
     //dump(req("q","{!scorejoin from="+toField+" to="+idField+" score=Max}title:random", "fl","id,score", "debug", "true"));
     /*
     Query joinQuery =
@@ -149,7 +149,7 @@ public class TestScoreJoinQPScore extends SolrTestCaseJ4 {
 
     // dump(req("q","title:movie", "fl","id,score", "debug", "true"));
     assertJQ(req("q", "{!join from=" + toField + " to=" + idField + " score=Max}title:movie", "fl", "id")
-        , "/response=={'numFound':2,'start':0,'docs':[{'id':'4'},{'id':'1'}]}");
+        , "/response=={'numFound':2,'start':0,'numFoundExact':true,'docs':[{'id':'4'},{'id':'1'}]}");
     
     /*joinQuery = JoinUtil.createJoinQuery(toField, false, idField, new TermQuery(new Term("title", "movie")), indexSearcher, ScoreMode.Max);
     result = indexSearcher.search(joinQuery, 10);
@@ -159,7 +159,7 @@ public class TestScoreJoinQPScore extends SolrTestCaseJ4 {
 
     // Score mode total
     assertJQ(req("q", "{!join from=" + toField + " to=" + idField + " score=Total}title:movie", "fl", "id")
-        , "/response=={'numFound':2,'start':0,'docs':[{'id':'1'},{'id':'4'}]}");
+        , "/response=={'numFound':2,'start':0,'numFoundExact':true,'docs':[{'id':'1'},{'id':'4'}]}");
   /*  joinQuery = JoinUtil.createJoinQuery(toField, false, idField, new TermQuery(new Term("title", "movie")), indexSearcher, ScoreMode.Total);
     result = indexSearcher.search(joinQuery, 10);
     assertEquals(2, result.totalHits);
@@ -168,7 +168,7 @@ public class TestScoreJoinQPScore extends SolrTestCaseJ4 {
 */
     //Score mode avg
     assertJQ(req("q", "{!join from=" + toField + " to=" + idField + " score=Avg}title:movie", "fl", "id")
-        , "/response=={'numFound':2,'start':0,'docs':[{'id':'4'},{'id':'1'}]}");
+        , "/response=={'numFound':2,'start':0,'numFoundExact':true,'docs':[{'id':'4'},{'id':'1'}]}");
     
   /*  joinQuery = JoinUtil.createJoinQuery(toField, false, idField, new TermQuery(new Term("title", "movie")), indexSearcher, ScoreMode.Avg);
     result = indexSearcher.search(joinQuery, 10);
diff --git a/solr/solrj/src/java/org/apache/solr/common/SolrDocumentList.java b/solr/solrj/src/java/org/apache/solr/common/SolrDocumentList.java
index 5875833..ab13b2d 100644
--- a/solr/solrj/src/java/org/apache/solr/common/SolrDocumentList.java
+++ b/solr/solrj/src/java/org/apache/solr/common/SolrDocumentList.java
@@ -31,7 +31,16 @@ public class SolrDocumentList extends ArrayList<SolrDocument>
   private long numFound = 0;
   private long start = 0;
   private Float maxScore = null;
+  private Boolean numFoundExact = true;
   
+  public Boolean getNumFoundExact() {
+    return numFoundExact;
+  }
+
+  public void setNumFoundExact(Boolean numFoundExact) {
+    this.numFoundExact = numFoundExact;
+  }
+
   public Float getMaxScore() {
     return maxScore;
   }
@@ -59,6 +68,7 @@ public class SolrDocumentList extends ArrayList<SolrDocument>
   @Override
   public String toString() {
     return "{numFound="+numFound
+            +",numFoundExact="+String.valueOf(numFoundExact)
             +",start="+start
             + (maxScore!=null ? ",maxScore="+maxScore : "")
             +",docs="+super.toString()
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java b/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
index 2261972..2f88c96 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
@@ -162,6 +162,12 @@ public interface CommonParams {
    * Timeout value in milliseconds.  If not set, or the value is &gt;= 0, there is no timeout.
    */
   String TIME_ALLOWED = "timeAllowed";
+
+  /**
+   * The number of hits that need to be counted accurately. If more than {@link #MIN_EXACT_HITS} documents
+   * match a query, then the value in "numFound" may be an estimate to speedup search.
+   */
+  String MIN_EXACT_HITS = "minExactHits";
   
   /** 'true' if the header should include the handler name */
   String HEADER_ECHO_HANDLER = "echoHandler";
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/FastJavaBinDecoder.java b/solr/solrj/src/java/org/apache/solr/common/util/FastJavaBinDecoder.java
index 1508494..84c35bd 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/FastJavaBinDecoder.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/FastJavaBinDecoder.java
@@ -479,6 +479,9 @@ public class FastJavaBinDecoder implements DataEntry.FastDecoder {
           solrDocs.setNumFound((Long) list.get(0));
           solrDocs.setStart((Long) list.get(1));
           solrDocs.setMaxScore((Float) list.get(2));
+          if (list.size() > 3) { //needed for back compatibility
+            solrDocs.setNumFoundExact((Boolean)list.get(3));
+          }
         }
         List<SolrDocument> l =  codec.readArray(codec.dis, entry.size);
         solrDocs.addAll(l);
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/JavaBinCodec.java b/solr/solrj/src/java/org/apache/solr/common/util/JavaBinCodec.java
index e50c742..bd86db9 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/JavaBinCodec.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/JavaBinCodec.java
@@ -588,10 +588,14 @@ public class JavaBinCodec implements PushWriter {
 
   public SolrDocumentList readSolrDocumentList(DataInputInputStream dis) throws IOException {
     SolrDocumentList solrDocs = new SolrDocumentList();
-    List list = (List) readVal(dis);
+    @SuppressWarnings("unchecked")
+    List<Object> list = (List<Object>) readVal(dis);
     solrDocs.setNumFound((Long) list.get(0));
     solrDocs.setStart((Long) list.get(1));
     solrDocs.setMaxScore((Float) list.get(2));
+    if (list.size() > 3) { //needed for back compatibility
+      solrDocs.setNumFoundExact((Boolean)list.get(3));
+    }
 
     @SuppressWarnings("unchecked")
     List<SolrDocument> l = (List<SolrDocument>) readVal(dis);
@@ -602,10 +606,11 @@ public class JavaBinCodec implements PushWriter {
   public void writeSolrDocumentList(SolrDocumentList docs)
           throws IOException {
     writeTag(SOLRDOCLST);
-    List<Number> l = new ArrayList<>(3);
+    List<Object> l = new ArrayList<>(4);
     l.add(docs.getNumFound());
     l.add(docs.getStart());
     l.add(docs.getMaxScore());
+    l.add(docs.getNumFoundExact());
     writeArray(l);
     writeArray(docs);
   }
diff --git a/solr/solrj/src/test-files/solrj/javabin_backcompat.bin b/solr/solrj/src/test-files/solrj/javabin_backcompat.bin
index 6e9d32f..7a5fde2 100644
Binary files a/solr/solrj/src/test-files/solrj/javabin_backcompat.bin and b/solr/solrj/src/test-files/solrj/javabin_backcompat.bin differ
diff --git a/solr/solrj/src/test/org/apache/solr/common/params/CommonParamsTest.java b/solr/solrj/src/test/org/apache/solr/common/params/CommonParamsTest.java
index 98e2368..24a09d6 100755
--- a/solr/solrj/src/test/org/apache/solr/common/params/CommonParamsTest.java
+++ b/solr/solrj/src/test/org/apache/solr/common/params/CommonParamsTest.java
@@ -33,4 +33,6 @@ public class CommonParamsTest extends SolrTestCase
   public void testRowsDefault() { assertEquals(10, CommonParams.ROWS_DEFAULT); }
 
   public void testPreferLocalShards() { assertEquals("preferLocalShards", CommonParams.PREFER_LOCAL_SHARDS); }
+  
+  public void testMinExactHits() { assertEquals("minExactHits", CommonParams.MIN_EXACT_HITS); }
 }
diff --git a/solr/solrj/src/test/org/apache/solr/common/util/TestJavaBinCodec.java b/solr/solrj/src/test/org/apache/solr/common/util/TestJavaBinCodec.java
index 7737f0c..d6cd3dc 100644
--- a/solr/solrj/src/test/org/apache/solr/common/util/TestJavaBinCodec.java
+++ b/solr/solrj/src/test/org/apache/solr/common/util/TestJavaBinCodec.java
@@ -16,6 +16,19 @@
  */
 package org.apache.solr.common.util;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.EnumFieldValue;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.SolrInputField;
+import org.apache.solr.util.ConcurrentLRUCache;
+import org.apache.solr.util.RTimer;
+import org.junit.Test;
+import org.noggit.CharArr;
+
 import java.io.BufferedOutputStream;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -31,19 +44,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 
-import org.apache.commons.io.IOUtils;
-import org.apache.lucene.util.TestUtil;
-import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.common.EnumFieldValue;
-import org.apache.solr.common.SolrDocument;
-import org.apache.solr.common.SolrDocumentList;
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.SolrInputField;
-import org.apache.solr.util.ConcurrentLRUCache;
-import org.apache.solr.util.RTimer;
-import org.junit.Test;
-import org.noggit.CharArr;
-
 public class TestJavaBinCodec extends SolrTestCaseJ4 {
 
   private static final String SOLRJ_JAVABIN_BACKCOMPAT_BIN = "/solrj/javabin_backcompat.bin";
@@ -137,6 +137,7 @@ public class TestJavaBinCodec extends SolrTestCaseJ4 {
     SolrDocumentList solrDocs = new SolrDocumentList();
     solrDocs.setMaxScore(1.0f);
     solrDocs.setNumFound(1);
+    solrDocs.setNumFoundExact(Boolean.TRUE);
     solrDocs.setStart(0);
     solrDocs.add(0, doc);
     types.add(solrDocs);
@@ -515,7 +516,7 @@ public class TestJavaBinCodec extends SolrTestCaseJ4 {
   }
 
   private static void runInThreads(int count,  Runnable runnable) throws InterruptedException {
-    ArrayList<Thread> t =new ArrayList();
+    ArrayList<Thread> t =new ArrayList<>();
     for(int i=0;i<count;i++ ) t.add(new Thread(runnable));
     for (Thread thread : t) thread.start();
     for (Thread thread : t) thread.join();
@@ -537,14 +538,14 @@ public class TestJavaBinCodec extends SolrTestCaseJ4 {
 
   }
 
-  public static void main(String[] args) {
-    // TestJavaBinCodec test = new TestJavaBinCodec();
-    // test.genBinaryFiles();
-    try {
-      doDecodePerf(args);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
+  public static void main(String[] args) throws IOException {
+     TestJavaBinCodec test = new TestJavaBinCodec();
+     test.genBinaryFiles();
+//    try {
+//      doDecodePerf(args);
+//    } catch (Exception e) {
+//      throw new RuntimeException(e);
+//    }
   }
 
   // common-case ascii