You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2017/01/17 13:32:37 UTC

[1/3] lucene-solr:branch_6x: LUCENE-7637: Require that all terms of a TermsQuery come from the same field.

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x 9224065cb -> 5c2f5bd02


LUCENE-7637: Require that all terms of a TermsQuery come from the same field.


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

Branch: refs/heads/branch_6x
Commit: 5c2f5bd028d370f43fbeee07fc6d197449609af3
Parents: aba6539
Author: Adrien Grand <jp...@gmail.com>
Authored: Tue Jan 17 14:03:46 2017 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Tue Jan 17 14:28:12 2017 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   8 +
 .../apache/lucene/search/TermInSetQuery.java    |  93 ++----
 .../lucene/search/TermInSetQueryTest.java       | 123 +++-----
 .../apache/lucene/facet/MultiFacetQuery.java    |  13 +-
 .../org/apache/lucene/queries/TermsQuery.java   |  71 ++++-
 .../apache/lucene/queries/TermsQueryTest.java   | 310 +++++++++++++++++++
 6 files changed, 458 insertions(+), 160 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5c2f5bd0/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 392c3f0..9743459 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -5,6 +5,14 @@ http://s.apache.org/luceneversions
 
 ======================= Lucene 6.5.0 =======================
 
+API Changes
+
+* LUCENE-7624: TermsQuery has been renamed as TermInSetQuery and moved to core.
+  (Alan Woodward)
+
+* LUCENE-7637: TermInSetQuery requires that all terms come from the same field.
+  (Adrien Grand)
+
 New Features
 
 * LUCENE-7623: Add FunctionScoreQuery and FunctionMatchQuery (Alan Woodward,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5c2f5bd0/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
index 978bd2d..100513b 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Objects;
 import java.util.Set;
@@ -73,39 +72,12 @@ public class TermInSetQuery extends Query implements Accountable {
   // Same threshold as MultiTermQueryConstantScoreWrapper
   static final int BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD = 16;
 
-  private final boolean singleField; // whether all terms are from the same field
+  private final String field;
   private final PrefixCodedTerms termData;
   private final int termDataHashCode; // cached hashcode of termData
 
   /**
-   * Creates a new {@link TermInSetQuery} from the given collection. It
-   * can contain duplicate terms and multiple fields.
-   */
-  public TermInSetQuery(Collection<Term> terms) {
-    Term[] sortedTerms = terms.toArray(new Term[terms.size()]);
-    // already sorted if we are a SortedSet with natural order
-    boolean sorted = terms instanceof SortedSet && ((SortedSet<Term>)terms).comparator() == null;
-    if (!sorted) {
-      ArrayUtil.timSort(sortedTerms);
-    }
-    PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
-    Set<String> fields = new HashSet<>();
-    Term previous = null;
-    for (Term term : sortedTerms) {
-      if (term.equals(previous) == false) {
-        fields.add(term.field());
-        builder.add(term);
-      }
-      previous = term;
-    }
-    singleField = fields.size() == 1;
-    termData = builder.finish();
-    termDataHashCode = termData.hashCode();
-  }
-
-  /**
-   * Creates a new {@link TermInSetQuery} from the given collection for
-   * a single field. It can contain duplicate terms.
+   * Creates a new {@link TermInSetQuery} from the given collection of terms.
    */
   public TermInSetQuery(String field, Collection<BytesRef> terms) {
     BytesRef[] sortedTerms = terms.toArray(new BytesRef[terms.size()]);
@@ -125,27 +97,18 @@ public class TermInSetQuery extends Query implements Accountable {
       builder.add(field, term);
       previous.copyBytes(term);
     }
-    singleField = true;
+    this.field = field;
     termData = builder.finish();
     termDataHashCode = termData.hashCode();
   }
 
   /**
-   * Creates a new {@link TermInSetQuery} from the given {@link BytesRef} array for
-   * a single field.
+   * Creates a new {@link TermInSetQuery} from the given array of terms.
    */
   public TermInSetQuery(String field, BytesRef...terms) {
     this(field, Arrays.asList(terms));
   }
 
-  /**
-   * Creates a new {@link TermInSetQuery} from the given array. The array can
-   * contain duplicate terms and multiple fields.
-   */
-  public TermInSetQuery(final Term... terms) {
-    this(Arrays.asList(terms));
-  }
-
   @Override
   public Query rewrite(IndexReader reader) throws IOException {
     final int threshold = Math.min(BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD, BooleanQuery.getMaxClauseCount());
@@ -167,6 +130,7 @@ public class TermInSetQuery extends Query implements Accountable {
   }
 
   private boolean equalsTo(TermInSetQuery other) {
+    // no need to check 'field' explicitly since it is encoded in 'termData'
     // termData might be heavy to compare so check the hash code first
     return termDataHashCode == other.termDataHashCode &&
         termData.equals(other.termData);
@@ -260,6 +224,15 @@ public class TermInSetQuery extends Query implements Accountable {
       private WeightOrDocIdSet rewrite(LeafReaderContext context) throws IOException {
         final LeafReader reader = context.reader();
 
+        final Fields fields = reader.fields();
+        Terms terms = fields.terms(field);
+        if (terms == null) {
+          return null;
+        }
+        TermsEnum termsEnum = terms.iterator();
+        PostingsEnum docs = null;
+        TermIterator iterator = termData.iterator();
+
         // We will first try to collect up to 'threshold' terms into 'matchingTerms'
         // if there are two many terms, we will fall back to building the 'builder'
         final int threshold = Math.min(BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD, BooleanQuery.getMaxClauseCount());
@@ -267,25 +240,9 @@ public class TermInSetQuery extends Query implements Accountable {
         List<TermAndState> matchingTerms = new ArrayList<>(threshold);
         DocIdSetBuilder builder = null;
 
-        final Fields fields = reader.fields();
-        String lastField = null;
-        Terms terms = null;
-        TermsEnum termsEnum = null;
-        PostingsEnum docs = null;
-        TermIterator iterator = termData.iterator();
         for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
-          String field = iterator.field();
-          // comparing references is fine here
-          if (field != lastField) {
-            terms = fields.terms(field);
-            if (terms == null) {
-              termsEnum = null;
-            } else {
-              termsEnum = terms.iterator();
-            }
-            lastField = field;
-          }
-          if (termsEnum != null && termsEnum.seekExact(term)) {
+          assert field.equals(iterator.field());
+          if (termsEnum.seekExact(term)) {
             if (matchingTerms == null) {
               docs = termsEnum.postings(docs, PostingsEnum.NONE);
               builder.add(docs);
@@ -293,15 +250,7 @@ public class TermInSetQuery extends Query implements Accountable {
               matchingTerms.add(new TermAndState(field, termsEnum));
             } else {
               assert matchingTerms.size() == threshold;
-              if (singleField) {
-                // common case: all terms are in the same field
-                // use an optimized builder that leverages terms stats to be more efficient
-                builder = new DocIdSetBuilder(reader.maxDoc(), terms);
-              } else {
-                // corner case: different fields
-                // don't make assumptions about the docs we will get
-                builder = new DocIdSetBuilder(reader.maxDoc());
-              }
+              builder = new DocIdSetBuilder(reader.maxDoc(), terms);
               docs = termsEnum.postings(docs, PostingsEnum.NONE);
               builder.add(docs);
               for (TermAndState t : matchingTerms) {
@@ -345,7 +294,9 @@ public class TermInSetQuery extends Query implements Accountable {
       @Override
       public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
         final WeightOrDocIdSet weightOrBitSet = rewrite(context);
-        if (weightOrBitSet.weight != null) {
+        if (weightOrBitSet == null) {
+          return null;
+        } else if (weightOrBitSet.weight != null) {
           return weightOrBitSet.weight.bulkScorer(context);
         } else {
           final Scorer scorer = scorer(weightOrBitSet.set);
@@ -359,7 +310,9 @@ public class TermInSetQuery extends Query implements Accountable {
       @Override
       public Scorer scorer(LeafReaderContext context) throws IOException {
         final WeightOrDocIdSet weightOrBitSet = rewrite(context);
-        if (weightOrBitSet.weight != null) {
+        if (weightOrBitSet == null) {
+          return null;
+        } else if (weightOrBitSet.weight != null) {
           return weightOrBitSet.weight.scorer(context);
         } else {
           return scorer(weightOrBitSet.set);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5c2f5bd0/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java b/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
index e694d97..3878d59 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
@@ -18,15 +18,12 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 import com.carrotsearch.randomizedtesting.generators.RandomStrings;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field.Store;
@@ -53,25 +50,25 @@ public class TermInSetQueryTest extends LuceneTestCase {
 
   public void testDuel() throws IOException {
     final int iters = atLeast(2);
+    final String field = "f";
     for (int iter = 0; iter < iters; ++iter) {
-      final List<Term> allTerms = new ArrayList<>();
+      final List<BytesRef> allTerms = new ArrayList<>();
       final int numTerms = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 10));
       for (int i = 0; i < numTerms; ++i) {
-        final String field = usually() ? "f" : "g";
         final String value = TestUtil.randomAnalysisString(random(), 10, true);
-        allTerms.add(new Term(field, value));
+        allTerms.add(new BytesRef(value));
       }
       Directory dir = newDirectory();
       RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
       final int numDocs = atLeast(100);
       for (int i = 0; i < numDocs; ++i) {
         Document doc = new Document();
-        final Term term = allTerms.get(random().nextInt(allTerms.size()));
-        doc.add(new StringField(term.field(), term.text(), Store.NO));
+        final BytesRef term = allTerms.get(random().nextInt(allTerms.size()));
+        doc.add(new StringField(field, term, Store.NO));
         iw.addDocument(doc);
       }
       if (numTerms > 1 && random().nextBoolean()) {
-        iw.deleteDocuments(new TermQuery(allTerms.get(0)));
+        iw.deleteDocuments(new TermQuery(new Term(field, allTerms.get(0))));
       }
       iw.commit();
       final IndexReader reader = iw.getReader();
@@ -87,16 +84,16 @@ public class TermInSetQueryTest extends LuceneTestCase {
       for (int i = 0; i < 100; ++i) {
         final float boost = random().nextFloat() * 10;
         final int numQueryTerms = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 8));
-        List<Term> queryTerms = new ArrayList<>();
+        List<BytesRef> queryTerms = new ArrayList<>();
         for (int j = 0; j < numQueryTerms; ++j) {
           queryTerms.add(allTerms.get(random().nextInt(allTerms.size())));
         }
         final BooleanQuery.Builder bq = new BooleanQuery.Builder();
-        for (Term t : queryTerms) {
-          bq.add(new TermQuery(t), Occur.SHOULD);
+        for (BytesRef t : queryTerms) {
+          bq.add(new TermQuery(new Term(field, t)), Occur.SHOULD);
         }
         final Query q1 = new ConstantScoreQuery(bq.build());
-        final Query q2 = new TermInSetQuery(queryTerms);
+        final Query q2 = new TermInSetQuery(field, queryTerms);
         assertSameMatches(searcher, new BoostQuery(q1, boost), new BoostQuery(q2, boost), true);
       }
 
@@ -118,103 +115,72 @@ public class TermInSetQueryTest extends LuceneTestCase {
     }
   }
 
-  private TermInSetQuery termsQuery(boolean singleField, Term...terms) {
-    return termsQuery(singleField, Arrays.asList(terms));
-  }
-
-  private TermInSetQuery termsQuery(boolean singleField, Collection<Term> termList) {
-    if (!singleField) {
-      return new TermInSetQuery(new ArrayList<>(termList));
-    }
-    final TermInSetQuery filter;
-    List<BytesRef> bytes = new ArrayList<>();
-    String field = null;
-    for (Term term : termList) {
-        bytes.add(term.bytes());
-        if (field != null) {
-          assertEquals(term.field(), field);
-        }
-        field = term.field();
-    }
-    assertNotNull(field);
-    filter = new TermInSetQuery(field, bytes);
-    return filter;
-  }
-
   public void testHashCodeAndEquals() {
     int num = atLeast(100);
-    final boolean singleField = random().nextBoolean();
-    List<Term> terms = new ArrayList<>();
-    Set<Term> uniqueTerms = new HashSet<>();
+    List<BytesRef> terms = new ArrayList<>();
+    Set<BytesRef> uniqueTerms = new HashSet<>();
     for (int i = 0; i < num; i++) {
-      String field = "field" + (singleField ? "1" : random().nextInt(100));
       String string = TestUtil.randomRealisticUnicodeString(random());
-      terms.add(new Term(field, string));
-      uniqueTerms.add(new Term(field, string));
-      TermInSetQuery left = termsQuery(singleField ? random().nextBoolean() : false, uniqueTerms);
+      terms.add(new BytesRef(string));
+      uniqueTerms.add(new BytesRef(string));
+      TermInSetQuery left = new TermInSetQuery("field", uniqueTerms);
       Collections.shuffle(terms, random());
-      TermInSetQuery right = termsQuery(singleField ? random().nextBoolean() : false, terms);
+      TermInSetQuery right = new TermInSetQuery("field", terms);
       assertEquals(right, left);
       assertEquals(right.hashCode(), left.hashCode());
       if (uniqueTerms.size() > 1) {
-        List<Term> asList = new ArrayList<>(uniqueTerms);
+        List<BytesRef> asList = new ArrayList<>(uniqueTerms);
         asList.remove(0);
-        TermInSetQuery notEqual = termsQuery(singleField ? random().nextBoolean() : false, asList);
+        TermInSetQuery notEqual = new TermInSetQuery("field", asList);
         assertFalse(left.equals(notEqual));
         assertFalse(right.equals(notEqual));
       }
     }
 
-    TermInSetQuery tq1 = new TermInSetQuery(new Term("thing", "apple"));
-    TermInSetQuery tq2 = new TermInSetQuery(new Term("thing", "orange"));
+    TermInSetQuery tq1 = new TermInSetQuery("thing", new BytesRef("apple"));
+    TermInSetQuery tq2 = new TermInSetQuery("thing", new BytesRef("orange"));
     assertFalse(tq1.hashCode() == tq2.hashCode());
 
     // different fields with the same term should have differing hashcodes
-    tq1 = new TermInSetQuery(new Term("thing1", "apple"));
-    tq2 = new TermInSetQuery(new Term("thing2", "apple"));
+    tq1 = new TermInSetQuery("thing", new BytesRef("apple"));
+    tq2 = new TermInSetQuery("thing2", new BytesRef("apple"));
     assertFalse(tq1.hashCode() == tq2.hashCode());
   }
 
-  public void testSingleFieldEquals() {
+  public void testSimpleEquals() {
     // Two terms with the same hash code
     assertEquals("AaAaBB".hashCode(), "BBBBBB".hashCode());
-    TermInSetQuery left = termsQuery(true, new Term("id", "AaAaAa"), new Term("id", "AaAaBB"));
-    TermInSetQuery right = termsQuery(true, new Term("id", "AaAaAa"), new Term("id", "BBBBBB"));
+    TermInSetQuery left = new TermInSetQuery("id", new BytesRef("AaAaAa"), new BytesRef("AaAaBB"));
+    TermInSetQuery right = new TermInSetQuery("id", new BytesRef("AaAaAa"), new BytesRef("BBBBBB"));
     assertFalse(left.equals(right));
   }
 
   public void testToString() {
-    TermInSetQuery termsQuery = new TermInSetQuery(new Term("field1", "a"),
-                                              new Term("field1", "b"),
-                                              new Term("field1", "c"));
+    TermInSetQuery termsQuery = new TermInSetQuery("field1",
+        new BytesRef("a"), new BytesRef("b"), new BytesRef("c"));
     assertEquals("field1:a field1:b field1:c", termsQuery.toString());
   }
 
   public void testDedup() {
-    Query query1 = new TermInSetQuery(new Term("foo", "bar"));
-    Query query2 = new TermInSetQuery(new Term("foo", "bar"), new Term("foo", "bar"));
+    Query query1 = new TermInSetQuery("foo", new BytesRef("bar"));
+    Query query2 = new TermInSetQuery("foo", new BytesRef("bar"), new BytesRef("bar"));
     QueryUtils.checkEqual(query1, query2);
   }
 
   public void testOrderDoesNotMatter() {
     // order of terms if different
-    Query query1 = new TermInSetQuery(new Term("foo", "bar"), new Term("foo", "baz"));
-    Query query2 = new TermInSetQuery(new Term("foo", "baz"), new Term("foo", "bar"));
-    QueryUtils.checkEqual(query1, query2);
-
-    // order of fields is different
-    query1 = new TermInSetQuery(new Term("foo", "bar"), new Term("bar", "bar"));
-    query2 = new TermInSetQuery(new Term("bar", "bar"), new Term("foo", "bar"));
+    Query query1 = new TermInSetQuery("foo", new BytesRef("bar"), new BytesRef("baz"));
+    Query query2 = new TermInSetQuery("foo", new BytesRef("baz"), new BytesRef("bar"));
     QueryUtils.checkEqual(query1, query2);
   }
 
   public void testRamBytesUsed() {
-    List<Term> terms = new ArrayList<>();
+    List<BytesRef> terms = new ArrayList<>();
     final int numTerms = 1000 + random().nextInt(1000);
     for (int i = 0; i < numTerms; ++i) {
-      terms.add(new Term("f", RandomStrings.randomUnicodeOfLength(random(), 10)));
+      terms.add(new BytesRef(RandomStrings.randomUnicodeOfLength(random(), 10)));
     }
-    TermInSetQuery query = new TermInSetQuery(terms);
+    TermInSetQuery query = new TermInSetQuery("f", terms);
     final long actualRamBytesUsed = RamUsageTester.sizeOf(query);
     final long expectedRamBytesUsed = query.ramBytesUsed();
     // error margin within 5%
@@ -281,43 +247,40 @@ public class TermInSetQueryTest extends LuceneTestCase {
 
   }
 
-  public void testPullOneTermsEnumPerField() throws Exception {
+  public void testPullOneTermsEnum() throws Exception {
     Directory dir = newDirectory();
     RandomIndexWriter w = new RandomIndexWriter(random(), dir);
     Document doc = new Document();
     doc.add(new StringField("foo", "1", Store.NO));
-    doc.add(new StringField("bar", "2", Store.NO));
-    doc.add(new StringField("baz", "3", Store.NO));
     w.addDocument(doc);
     DirectoryReader reader = w.getReader();
     w.close();
     final AtomicInteger counter = new AtomicInteger();
     DirectoryReader wrapped = new TermsCountingDirectoryReaderWrapper(reader, counter);
 
-    final List<Term> terms = new ArrayList<>();
-    final Set<String> fields = new HashSet<>();
+    final List<BytesRef> terms = new ArrayList<>();
     // enough terms to avoid the rewrite
     final int numTerms = TestUtil.nextInt(random(), TermInSetQuery.BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD + 1, 100);
     for (int i = 0; i < numTerms; ++i) {
-      final String field = RandomPicks.randomFrom(random(), new String[] {"foo", "bar", "baz"});
       final BytesRef term = new BytesRef(RandomStrings.randomUnicodeOfCodepointLength(random(), 10));
-      fields.add(field);
-      terms.add(new Term(field, term));
+      terms.add(term);
     }
 
-    new IndexSearcher(wrapped).count(new TermInSetQuery(terms));
-    assertEquals(fields.size(), counter.get());
+    assertEquals(0, new IndexSearcher(wrapped).count(new TermInSetQuery("bar", terms)));
+    assertEquals(0, counter.get()); // missing field
+    new IndexSearcher(wrapped).count(new TermInSetQuery("foo", terms));
+    assertEquals(1, counter.get());
     wrapped.close();
     dir.close();
   }
   
   public void testBinaryToString() {
-    TermInSetQuery query = new TermInSetQuery(new Term("field", new BytesRef(new byte[] { (byte) 0xff, (byte) 0xfe })));
+    TermInSetQuery query = new TermInSetQuery("field", new BytesRef(new byte[] { (byte) 0xff, (byte) 0xfe }));
     assertEquals("field:[ff fe]", query.toString());
   }
 
   public void testIsConsideredCostlyByQueryCache() throws IOException {
-    TermInSetQuery query = new TermInSetQuery(new Term("foo", "bar"), new Term("foo", "baz"));
+    TermInSetQuery query = new TermInSetQuery("foo", new BytesRef("bar"), new BytesRef("baz"));
     UsageTrackingQueryCachingPolicy policy = new UsageTrackingQueryCachingPolicy();
     assertFalse(policy.shouldCache(query));
     policy.onUse(query);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5c2f5bd0/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java b/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
index a010709..72c2773 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
@@ -19,9 +19,9 @@ package org.apache.lucene.facet;
 import java.util.ArrayList;
 import java.util.Collection;
 
-import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermInSetQuery;
+import org.apache.lucene.util.BytesRef;
 
 /**
  * A multi-terms {@link Query} over a {@link FacetField}.
@@ -38,7 +38,7 @@ public class MultiFacetQuery extends TermInSetQuery {
    * Creates a new {@code MultiFacetQuery} filtering the query on the given dimension.
    */
   public MultiFacetQuery(final FacetsConfig facetsConfig, final String dimension, final String[]... paths) {
-    super(toTerms(facetsConfig.getDimConfig(dimension), dimension, paths));
+    super(facetsConfig.getDimConfig(dimension).indexFieldName, toTerms(dimension, paths));
   }
 
   /**
@@ -47,14 +47,13 @@ public class MultiFacetQuery extends TermInSetQuery {
    * <b>NOTE:</b>Uses FacetsConfig.DEFAULT_DIM_CONFIG.
    */
   public MultiFacetQuery(final String dimension, final String[]... paths) {
-    super(toTerms(FacetsConfig.DEFAULT_DIM_CONFIG, dimension, paths));
+    super(FacetsConfig.DEFAULT_DIM_CONFIG.indexFieldName, toTerms(dimension, paths));
   }
 
-  static Collection<Term> toTerms(final FacetsConfig.DimConfig dimConfig, final String dimension,
-          final String[]... paths) {
-    final Collection<Term> terms = new ArrayList<>(paths.length);
+  static Collection<BytesRef> toTerms(final String dimension, final String[]... paths) {
+    final Collection<BytesRef> terms = new ArrayList<>(paths.length);
     for (String[] path : paths)
-      terms.add(FacetQuery.toTerm(dimConfig, dimension, path));
+      terms.add(new BytesRef(FacetsConfig.pathToString(dimension, path)));
     return terms;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5c2f5bd0/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
index 994c60f..9b45a9c 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
@@ -16,25 +16,61 @@
  */
 package org.apache.lucene.queries;
 
+import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
 
+import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermInSetQuery;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
 
 /**
  * @deprecated Use {@link org.apache.lucene.search.TermInSetQuery}
  */
 @Deprecated
-public class TermsQuery extends TermInSetQuery {
+public class TermsQuery extends Query implements Accountable {
+
+  private final Query rewritten;
+  private final long ramBytesUsed;
 
   /**
    * Creates a new {@link TermsQuery} from the given collection. It
    * can contain duplicate terms and multiple fields.
    */
   public TermsQuery(Collection<Term> terms) {
-    super(terms);
+    Map<String, List<BytesRef>> termsPerField = new HashMap<>();
+    for (Term term : terms) {
+      List<BytesRef> t = termsPerField.computeIfAbsent(term.field(), s -> new ArrayList<>());
+      t.add(term.bytes());
+    }
+    if (termsPerField.size() == 1) {
+      Map.Entry<String, List<BytesRef>> entry = termsPerField.entrySet().iterator().next();
+      TermInSetQuery tisq = new TermInSetQuery(entry.getKey(), entry.getValue());
+      rewritten = tisq;
+      ramBytesUsed = tisq.ramBytesUsed();
+    } else {
+      BooleanQuery.Builder bq = new BooleanQuery.Builder()
+          .setDisableCoord(true);
+      long ramBytesUsed = 0;
+      for (Map.Entry<String, List<BytesRef>> entry : termsPerField.entrySet()) {
+        TermInSetQuery tisq = new TermInSetQuery(entry.getKey(), entry.getValue());
+        bq.add(tisq, Occur.SHOULD);
+        ramBytesUsed += tisq.ramBytesUsed();
+      }
+      rewritten = new ConstantScoreQuery(bq.build());
+      this.ramBytesUsed = ramBytesUsed;
+    }
   }
 
   /**
@@ -42,7 +78,9 @@ public class TermsQuery extends TermInSetQuery {
    * a single field. It can contain duplicate terms.
    */
   public TermsQuery(String field, Collection<BytesRef> terms) {
-    super(field, terms);
+    TermInSetQuery tisq = new TermInSetQuery(field, terms);
+    rewritten = tisq;
+    ramBytesUsed = tisq.ramBytesUsed();
   }
 
   /**
@@ -61,4 +99,31 @@ public class TermsQuery extends TermInSetQuery {
     this(Arrays.asList(terms));
   }
 
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    return rewritten;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (sameClassAs(obj) == false) {
+      return false;
+    }
+    return Objects.equals(rewritten, ((TermsQuery) obj).rewritten);
+  }
+
+  @Override
+  public int hashCode() {
+    return 31 * classHash() + rewritten.hashCode();
+  }
+
+  @Override
+  public String toString(String field) {
+    return rewritten.toString(field);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return ramBytesUsed;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5c2f5bd0/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java b/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java
new file mode 100644
index 0000000..c5549da
--- /dev/null
+++ b/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java
@@ -0,0 +1,310 @@
+/*
+ * 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.lucene.queries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.FilterDirectoryReader;
+import org.apache.lucene.index.FilterLeafReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.MultiReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.BoostQuery;
+import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryUtils;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.UsageTrackingQueryCachingPolicy;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.RamUsageTester;
+import org.apache.lucene.util.TestUtil;
+
+import com.carrotsearch.randomizedtesting.generators.RandomStrings;
+
+public class TermsQueryTest extends LuceneTestCase {
+
+  public void testDuel() throws IOException {
+    final int iters = atLeast(2);
+    for (int iter = 0; iter < iters; ++iter) {
+      final List<Term> allTerms = new ArrayList<>();
+      final int numTerms = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 10));
+      for (int i = 0; i < numTerms; ++i) {
+        final String field = usually() ? "f" : "g";
+        final String value = TestUtil.randomAnalysisString(random(), 10, true);
+        allTerms.add(new Term(field, value));
+      }
+      Directory dir = newDirectory();
+      RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+      final int numDocs = atLeast(100);
+      for (int i = 0; i < numDocs; ++i) {
+        Document doc = new Document();
+        final Term term = allTerms.get(random().nextInt(allTerms.size()));
+        doc.add(new StringField(term.field(), term.text(), Store.NO));
+        iw.addDocument(doc);
+      }
+      if (numTerms > 1 && random().nextBoolean()) {
+        iw.deleteDocuments(new TermQuery(allTerms.get(0)));
+      }
+      iw.commit();
+      final IndexReader reader = iw.getReader();
+      final IndexSearcher searcher = newSearcher(reader);
+      iw.close();
+
+      if (reader.numDocs() == 0) {
+        // may occasionally happen if all documents got the same term
+        IOUtils.close(reader, dir);
+        continue;
+      }
+
+      for (int i = 0; i < 100; ++i) {
+        final float boost = random().nextFloat() * 10;
+        final int numQueryTerms = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 8));
+        List<Term> queryTerms = new ArrayList<>();
+        for (int j = 0; j < numQueryTerms; ++j) {
+          queryTerms.add(allTerms.get(random().nextInt(allTerms.size())));
+        }
+        final BooleanQuery.Builder bq = new BooleanQuery.Builder();
+        for (Term t : queryTerms) {
+          bq.add(new TermQuery(t), Occur.SHOULD);
+        }
+        final Query q1 = new ConstantScoreQuery(bq.build());
+        final Query q2 = new TermsQuery(queryTerms);
+        assertSameMatches(searcher, new BoostQuery(q1, boost), new BoostQuery(q2, boost), true);
+      }
+
+      reader.close();
+      dir.close();
+    }
+  }
+
+  private void assertSameMatches(IndexSearcher searcher, Query q1, Query q2, boolean scores) throws IOException {
+    final int maxDoc = searcher.getIndexReader().maxDoc();
+    final TopDocs td1 = searcher.search(q1, maxDoc, scores ? Sort.RELEVANCE : Sort.INDEXORDER);
+    final TopDocs td2 = searcher.search(q2, maxDoc, scores ? Sort.RELEVANCE : Sort.INDEXORDER);
+    assertEquals(td1.totalHits, td2.totalHits);
+    for (int i = 0; i < td1.scoreDocs.length; ++i) {
+      assertEquals(td1.scoreDocs[i].doc, td2.scoreDocs[i].doc);
+      if (scores) {
+        assertEquals(td1.scoreDocs[i].score, td2.scoreDocs[i].score, 10e-7);
+      }
+    }
+  }
+
+  private TermsQuery termsQuery(boolean singleField, Term...terms) {
+    return termsQuery(singleField, Arrays.asList(terms));
+  }
+
+  private TermsQuery termsQuery(boolean singleField, Collection<Term> termList) {
+    if (!singleField) {
+      return new TermsQuery(new ArrayList<>(termList));
+    }
+    final TermsQuery filter;
+    List<BytesRef> bytes = new ArrayList<>();
+    String field = null;
+    for (Term term : termList) {
+        bytes.add(term.bytes());
+        if (field != null) {
+          assertEquals(term.field(), field);
+        }
+        field = term.field();
+    }
+    assertNotNull(field);
+    filter = new TermsQuery(field, bytes);
+    return filter;
+  }
+
+  public void testHashCodeAndEquals() {
+    int num = atLeast(100);
+    final boolean singleField = random().nextBoolean();
+    List<Term> terms = new ArrayList<>();
+    Set<Term> uniqueTerms = new HashSet<>();
+    for (int i = 0; i < num; i++) {
+      String field = "field" + (singleField ? "1" : random().nextInt(100));
+      String string = TestUtil.randomRealisticUnicodeString(random());
+      terms.add(new Term(field, string));
+      uniqueTerms.add(new Term(field, string));
+      TermsQuery left = termsQuery(singleField ? random().nextBoolean() : false, uniqueTerms);
+      Collections.shuffle(terms, random());
+      TermsQuery right = termsQuery(singleField ? random().nextBoolean() : false, terms);
+      assertEquals(right, left);
+      assertEquals(right.hashCode(), left.hashCode());
+      if (uniqueTerms.size() > 1) {
+        List<Term> asList = new ArrayList<>(uniqueTerms);
+        asList.remove(0);
+        TermsQuery notEqual = termsQuery(singleField ? random().nextBoolean() : false, asList);
+        assertFalse(left.equals(notEqual));
+        assertFalse(right.equals(notEqual));
+      }
+    }
+
+    TermsQuery tq1 = new TermsQuery(new Term("thing", "apple"));
+    TermsQuery tq2 = new TermsQuery(new Term("thing", "orange"));
+    assertFalse(tq1.hashCode() == tq2.hashCode());
+
+    // different fields with the same term should have differing hashcodes
+    tq1 = new TermsQuery(new Term("thing1", "apple"));
+    tq2 = new TermsQuery(new Term("thing2", "apple"));
+    assertFalse(tq1.hashCode() == tq2.hashCode());
+  }
+
+  public void testSingleFieldEquals() {
+    // Two terms with the same hash code
+    assertEquals("AaAaBB".hashCode(), "BBBBBB".hashCode());
+    TermsQuery left = termsQuery(true, new Term("id", "AaAaAa"), new Term("id", "AaAaBB"));
+    TermsQuery right = termsQuery(true, new Term("id", "AaAaAa"), new Term("id", "BBBBBB"));
+    assertFalse(left.equals(right));
+  }
+
+  public void testToString() {
+    TermsQuery termsQuery = new TermsQuery(new Term("field1", "a"),
+                                              new Term("field1", "b"),
+                                              new Term("field1", "c"));
+    assertEquals("field1:a field1:b field1:c", termsQuery.toString());
+  }
+
+  public void testDedup() {
+    Query query1 = new TermsQuery(new Term("foo", "bar"));
+    Query query2 = new TermsQuery(new Term("foo", "bar"), new Term("foo", "bar"));
+    QueryUtils.checkEqual(query1, query2);
+  }
+
+  public void testOrderDoesNotMatter() {
+    // order of terms if different
+    Query query1 = new TermsQuery(new Term("foo", "bar"), new Term("foo", "baz"));
+    Query query2 = new TermsQuery(new Term("foo", "baz"), new Term("foo", "bar"));
+    QueryUtils.checkEqual(query1, query2);
+
+    // order of fields is different
+    query1 = new TermsQuery(new Term("foo", "bar"), new Term("bar", "bar"));
+    query2 = new TermsQuery(new Term("bar", "bar"), new Term("foo", "bar"));
+    QueryUtils.checkEqual(query1, query2);
+  }
+
+  public void testRamBytesUsed() {
+    List<Term> terms = new ArrayList<>();
+    final int numTerms = 1000 + random().nextInt(1000);
+    for (int i = 0; i < numTerms; ++i) {
+      terms.add(new Term("f", RandomStrings.randomUnicodeOfLength(random(), 10)));
+    }
+    TermsQuery query = new TermsQuery(terms);
+    final long actualRamBytesUsed = RamUsageTester.sizeOf(query);
+    final long expectedRamBytesUsed = query.ramBytesUsed();
+    // error margin within 5%
+    assertEquals(actualRamBytesUsed, expectedRamBytesUsed, actualRamBytesUsed / 20);
+  }
+
+  private static class TermsCountingDirectoryReaderWrapper extends FilterDirectoryReader {
+
+    private final AtomicInteger counter;
+    
+    public TermsCountingDirectoryReaderWrapper(DirectoryReader in, AtomicInteger counter) throws IOException {
+      super(in, new TermsCountingSubReaderWrapper(counter));
+      this.counter = counter;
+    }
+
+    private static class TermsCountingSubReaderWrapper extends SubReaderWrapper {
+      private final AtomicInteger counter;
+
+      public TermsCountingSubReaderWrapper(AtomicInteger counter) {
+        this.counter = counter;
+      }
+
+      @Override
+      public LeafReader wrap(LeafReader reader) {
+        return new TermsCountingLeafReaderWrapper(reader, counter);
+      }
+    }
+
+    private static class TermsCountingLeafReaderWrapper extends FilterLeafReader {
+
+      private final AtomicInteger counter;
+
+      public TermsCountingLeafReaderWrapper(LeafReader in, AtomicInteger counter) {
+        super(in);
+        this.counter = counter;
+      }
+
+      @Override
+      public Fields fields() throws IOException {
+        return new FilterFields(in.fields()) {
+          @Override
+          public Terms terms(String field) throws IOException {
+            final Terms in = this.in.terms(field);
+            if (in == null) {
+              return null;
+            }
+            return new FilterTerms(in) {
+              @Override
+              public TermsEnum iterator() throws IOException {
+                counter.incrementAndGet();
+                return super.iterator();
+              }
+            };
+          }
+        };
+      }
+      
+    }
+
+    @Override
+    protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
+      return new TermsCountingDirectoryReaderWrapper(in, counter);
+    }
+
+  }
+  
+  public void testBinaryToString() {
+    TermsQuery query = new TermsQuery(new Term("field", new BytesRef(new byte[] { (byte) 0xff, (byte) 0xfe })));
+    assertEquals("field:[ff fe]", query.toString());
+  }
+
+  public void testIsConsideredCostlyByQueryCache() throws IOException {
+    Query query = new TermsQuery(new Term("foo", "bar"), new Term("foo", "baz"));
+    query = query.rewrite(new MultiReader());
+    UsageTrackingQueryCachingPolicy policy = new UsageTrackingQueryCachingPolicy();
+    assertFalse(policy.shouldCache(query));
+    policy.onUse(query);
+    policy.onUse(query);
+    // cached after two uses
+    assertTrue(policy.shouldCache(query));
+  }
+}


[2/3] lucene-solr:branch_6x: LUCENE-7055: Add ScorerProvider to get an estimation of the cost of scorers before building them.

Posted by jp...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/sandbox/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java
new file mode 100644
index 0000000..6ac980a
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java
@@ -0,0 +1,116 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+
+/**
+ * A query that uses either an index (points or terms) or doc values in order
+ * to run a range query, depending which one is more efficient.
+ */
+public final class IndexOrDocValuesQuery extends Query {
+
+  private final Query indexQuery, dvQuery;
+
+  /**
+   * Constructor that takes both a query that executes on an index structure
+   * like the inverted index or the points tree, and another query that
+   * executes on doc values. Both queries must match the same documents and
+   * attribute constant scores.
+   */
+  public IndexOrDocValuesQuery(Query indexQuery, Query dvQuery) {
+    this.indexQuery = indexQuery;
+    this.dvQuery = dvQuery;
+  }
+
+  @Override
+  public String toString(String field) {
+    return indexQuery.toString(field);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (sameClassAs(obj) == false) {
+      return false;
+    }
+    IndexOrDocValuesQuery that = (IndexOrDocValuesQuery) obj;
+    return indexQuery.equals(that.indexQuery) && dvQuery.equals(that.dvQuery);
+  }
+
+  @Override
+  public int hashCode() {
+    int h = classHash();
+    h = 31 * h + indexQuery.hashCode();
+    h = 31 * h + dvQuery.hashCode();
+    return h;
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    Query indexRewrite = indexQuery.rewrite(reader);
+    Query dvRewrite = dvQuery.rewrite(reader);
+    if (indexQuery != indexRewrite || dvQuery != dvRewrite) {
+      return new IndexOrDocValuesQuery(indexRewrite, dvRewrite);
+    }
+    return this;
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+    final Weight indexWeight = indexQuery.createWeight(searcher, needsScores);
+    final Weight dvWeight = dvQuery.createWeight(searcher, needsScores);
+    return new ConstantScoreWeight(this) {
+      @Override
+      public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
+        return indexWeight.bulkScorer(context);
+      }
+
+      @Override
+      public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
+        final ScorerSupplier indexScorerSupplier = indexWeight.scorerSupplier(context);
+        final ScorerSupplier dvScorerSupplier = dvWeight.scorerSupplier(context); 
+        if (indexScorerSupplier == null || dvScorerSupplier == null) {
+          return null;
+        }
+        return new ScorerSupplier() {
+          @Override
+          public Scorer get(boolean randomAccess) throws IOException {
+            return (randomAccess ? dvScorerSupplier : indexScorerSupplier).get(randomAccess);
+          }
+
+          @Override
+          public long cost() {
+            return Math.min(indexScorerSupplier.cost(), dvScorerSupplier.cost());
+          }
+        };
+      }
+
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        ScorerSupplier scorerSupplier = scorerSupplier(context);
+        if (scorerSupplier == null) {
+          return null;
+        }
+        return scorerSupplier.get(false);
+      }
+    };
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/sandbox/src/test/org/apache/lucene/search/TestIndexOrDocValuesQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestIndexOrDocValuesQuery.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestIndexOrDocValuesQuery.java
new file mode 100644
index 0000000..2a16e5d
--- /dev/null
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestIndexOrDocValuesQuery.java
@@ -0,0 +1,89 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+public class TestIndexOrDocValuesQuery extends LuceneTestCase {
+
+  public void testUseIndexForSelectiveQueries() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig()
+        // relies on costs and PointValues.estimateCost so we need the default codec
+        .setCodec(TestUtil.getDefaultCodec()));
+    for (int i = 0; i < 2000; ++i) {
+      Document doc = new Document();
+      if (i == 42) {
+        doc.add(new StringField("f1", "bar", Store.NO));
+        doc.add(new LongPoint("f2", 42L));
+        doc.add(new NumericDocValuesField("f2", 42L));
+      } else if (i == 100) {
+        doc.add(new StringField("f1", "foo", Store.NO));
+        doc.add(new LongPoint("f2", 2L));
+        doc.add(new NumericDocValuesField("f2", 2L));
+      } else {
+        doc.add(new StringField("f1", "bar", Store.NO));
+        doc.add(new LongPoint("f2", 2L));
+        doc.add(new NumericDocValuesField("f2", 2L));
+      }
+      w.addDocument(doc);
+    }
+    w.forceMerge(1);
+    IndexReader reader = DirectoryReader.open(w);
+    IndexSearcher searcher = newSearcher(reader);
+    searcher.setQueryCache(null);
+
+    // The term query is more selective, so the IndexOrDocValuesQuery should use doc values
+    final Query q1 = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("f1", "foo")), Occur.MUST)
+        .add(new IndexOrDocValuesQuery(LongPoint.newExactQuery("f2", 2), new DocValuesNumbersQuery("f2", 2L)), Occur.MUST)
+        .build();
+
+    final Weight w1 = searcher.createNormalizedWeight(q1, random().nextBoolean());
+    final Scorer s1 = w1.scorer(reader.leaves().get(0));
+    assertNotNull(s1.twoPhaseIterator()); // means we use doc values
+
+    // The term query is less selective, so the IndexOrDocValuesQuery should use points
+    final Query q2 = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("f1", "bar")), Occur.MUST)
+        .add(new IndexOrDocValuesQuery(LongPoint.newExactQuery("f2", 42), new DocValuesNumbersQuery("f2", 42L)), Occur.MUST)
+        .build();
+
+    final Weight w2 = searcher.createNormalizedWeight(q2, random().nextBoolean());
+    final Scorer s2 = w2.scorer(reader.leaves().get(0));
+    assertNull(s2.twoPhaseIterator()); // means we use points
+
+    reader.close();
+    w.close();
+    dir.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
index c6f5485..c397129 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
@@ -175,6 +175,13 @@ public final class AssertingPointsFormat extends PointsFormat {
     }
 
     @Override
+    public long estimatePointCount(String fieldName, IntersectVisitor visitor) {
+      final long value = in.estimatePointCount(fieldName, visitor);
+      assert value >= 0;
+      return value;
+    }
+
+    @Override
     public long ramBytesUsed() {
       long v = in.ramBytesUsed();
       assert v >= 0;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
index fd2260b..6491b40 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
@@ -125,6 +125,11 @@ class CrankyPointsFormat extends PointsFormat {
     }
 
     @Override
+    public long estimatePointCount(String fieldName, IntersectVisitor visitor) {
+      return delegate.estimatePointCount(fieldName, visitor);
+    }
+
+    @Override
     public byte[] getMinPackedValue(String fieldName) throws IOException {
       if (random.nextInt(100) == 0) {
         throw new IOException("Fake IOException");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java
index 75529df..7b6727d 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java
@@ -33,9 +33,45 @@ class AssertingWeight extends FilterWeight {
 
   @Override
   public Scorer scorer(LeafReaderContext context) throws IOException {
-    final Scorer inScorer = in.scorer(context);
-    assert inScorer == null || inScorer.docID() == -1;
-    return AssertingScorer.wrap(new Random(random.nextLong()), inScorer, needsScores);
+    if (random.nextBoolean()) {
+      final Scorer inScorer = in.scorer(context);
+      assert inScorer == null || inScorer.docID() == -1;
+      return AssertingScorer.wrap(new Random(random.nextLong()), inScorer, needsScores);
+    } else {
+      final ScorerSupplier scorerSupplier = scorerSupplier(context);
+      if (scorerSupplier == null) {
+        return null;
+      }
+      if (random.nextBoolean()) {
+        // Evil: make sure computing the cost has no side effects
+        scorerSupplier.cost();
+      }
+      return scorerSupplier.get(false);
+    }
+  }
+
+  @Override
+  public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
+    final ScorerSupplier inScorerSupplier = in.scorerSupplier(context);
+    if (inScorerSupplier == null) {
+      return null;
+    }
+    return new ScorerSupplier() {
+      private boolean getCalled = false;
+      @Override
+      public Scorer get(boolean randomAccess) throws IOException {
+        assert getCalled == false;
+        getCalled = true;
+        return AssertingScorer.wrap(new Random(random.nextLong()), inScorerSupplier.get(randomAccess), needsScores);
+      }
+
+      @Override
+      public long cost() {
+        final long cost = inScorerSupplier.cost();
+        assert cost >= 0;
+        return cost;
+      }
+    };
   }
 
   @Override


[3/3] lucene-solr:branch_6x: LUCENE-7055: Add ScorerProvider to get an estimation of the cost of scorers before building them.

Posted by jp...@apache.org.
LUCENE-7055: Add ScorerProvider to get an estimation of the cost of scorers before building them.


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

Branch: refs/heads/branch_6x
Commit: aba653960bc631c775128d31fcbb83e685b48680
Parents: 9224065
Author: Adrien Grand <jp...@gmail.com>
Authored: Mon Jan 16 15:47:53 2017 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Tue Jan 17 14:28:12 2017 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  10 +-
 .../codecs/simpletext/SimpleTextBKDReader.java  |  49 +++
 .../simpletext/SimpleTextPointsReader.java      |  11 +
 .../org/apache/lucene/codecs/PointsFormat.java  |   5 +
 .../org/apache/lucene/codecs/PointsWriter.java  |   5 +
 .../codecs/lucene60/Lucene60PointsReader.java   |  13 +
 .../org/apache/lucene/index/CheckIndex.java     |  37 ++
 .../apache/lucene/index/ParallelLeafReader.java |  13 +
 .../org/apache/lucene/index/PointValues.java    |   7 +
 .../apache/lucene/index/PointValuesWriter.java  |   6 +
 .../lucene/index/SlowCodecReaderWrapper.java    |   5 +
 .../apache/lucene/index/SortingLeafReader.java  |   5 +
 .../lucene/search/Boolean2ScorerSupplier.java   | 265 +++++++++++++++
 .../lucene/search/BooleanTopLevelScorers.java   |   2 +-
 .../org/apache/lucene/search/BooleanWeight.java | 179 ++--------
 .../apache/lucene/search/ConjunctionDISI.java   |   2 +-
 .../apache/lucene/search/ConjunctionScorer.java |   7 +-
 .../lucene/search/ConstantScoreQuery.java       |  46 ++-
 .../lucene/search/MinShouldMatchSumScorer.java  |  22 +-
 .../apache/lucene/search/PointRangeQuery.java   | 144 +++++---
 .../apache/lucene/search/ScorerSupplier.java    |  47 +++
 .../java/org/apache/lucene/search/Weight.java   |  25 ++
 .../org/apache/lucene/util/bkd/BKDReader.java   |  57 ++++
 .../search/TestBoolean2ScorerSupplier.java      | 339 +++++++++++++++++++
 .../search/TestBooleanQueryVisitSubscorers.java |   4 +-
 .../lucene/search/TestConjunctionDISI.java      |   2 +-
 .../apache/lucene/search/TestFilterWeight.java  |   3 +-
 .../apache/lucene/util/TestDocIdSetBuilder.java |   5 +
 .../apache/lucene/util/bkd/Test2BBKDPoints.java |   6 +
 .../util/bkd/TestMutablePointsReaderUtils.java  |   5 +
 .../apache/lucene/index/memory/MemoryIndex.java |   5 +
 .../lucene/search/DocValuesRangeQuery.java      |  11 +-
 .../lucene/search/IndexOrDocValuesQuery.java    | 116 +++++++
 .../search/TestIndexOrDocValuesQuery.java       |  89 +++++
 .../codecs/asserting/AssertingPointsFormat.java |   7 +
 .../codecs/cranky/CrankyPointsFormat.java       |   5 +
 .../apache/lucene/search/AssertingWeight.java   |  42 ++-
 37 files changed, 1351 insertions(+), 250 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 8b154ce..392c3f0 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -15,6 +15,13 @@ Bug Fixes
 * LUCENE-7630: Fix (Edge)NGramTokenFilter to no longer drop payloads
   and preserve all attributes. (Nathan Gass via Uwe Schindler)
 
+Improvements
+
+* LUCENE-7055: Added Weight#scorerSupplier, which allows to estimate the cost
+  of a Scorer before actually building it, in order to optimize how the query
+  should be run, eg. using points or doc values depending on costs of other
+  parts of the query. (Adrien Grand)
+
 ======================= Lucene 6.4.0 =======================
 
 API Changes
@@ -42,9 +49,6 @@ API Changes
 * LUCENE-7611: DocumentValueSourceDictionary now takes a LongValuesSource
   as a parameter, and the ValueSource equivalent is deprecated (Alan Woodward)
 
-* LUCENE-7624: TermsQuery has been renamed as TermInSetQuery and moved to core.
-  (Alan Woodward)
-
 New features
 
 * LUCENE-5867: Added BooleanSimilarity. (Robert Muir, Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDReader.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDReader.java
index bea7b62..c6085e7 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDReader.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDReader.java
@@ -287,6 +287,55 @@ final class SimpleTextBKDReader implements Accountable {
     }
   }
 
+  public long estimatePointCount(IntersectVisitor visitor) {
+    return estimatePointCount(getIntersectState(visitor), 1, minPackedValue, maxPackedValue);
+  }
+
+  private long estimatePointCount(IntersectState state,
+      int nodeID, byte[] cellMinPacked, byte[] cellMaxPacked) {
+    Relation r = state.visitor.compare(cellMinPacked, cellMaxPacked);
+
+    if (r == Relation.CELL_OUTSIDE_QUERY) {
+      // This cell is fully outside of the query shape: stop recursing
+      return 0L;
+    } else if (nodeID >= leafNodeOffset) {
+      // Assume all points match and there are no dups
+      return maxPointsInLeafNode;
+    } else {
+      
+      // Non-leaf node: recurse on the split left and right nodes
+
+      int address = nodeID * bytesPerIndexEntry;
+      int splitDim;
+      if (numDims == 1) {
+        splitDim = 0;
+      } else {
+        splitDim = splitPackedValues[address++] & 0xff;
+      }
+      
+      assert splitDim < numDims;
+
+      // TODO: can we alloc & reuse this up front?
+
+      byte[] splitPackedValue = new byte[packedBytesLength];
+
+      // Recurse on left sub-tree:
+      System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedBytesLength);
+      System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
+      final long leftCost = estimatePointCount(state,
+                2*nodeID,
+                cellMinPacked, splitPackedValue);
+
+      // Recurse on right sub-tree:
+      System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedBytesLength);
+      System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
+      final long rightCost = estimatePointCount(state,
+                2*nodeID+1,
+                splitPackedValue, cellMaxPacked);
+      return leftCost + rightCost;
+    }
+  }
+
   /** Copies the split value for this node into the provided byte array */
   public void copySplitValue(int nodeID, byte[] splitPackedValue) {
     int address = nodeID * bytesPerIndexEntry;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsReader.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsReader.java
index e6711e7..464e825 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsReader.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsReader.java
@@ -208,6 +208,17 @@ class SimpleTextPointsReader extends PointsReader {
   }
 
   @Override
+  public long estimatePointCount(String fieldName, IntersectVisitor visitor) {
+    SimpleTextBKDReader bkdReader = getBKDReader(fieldName);
+    if (bkdReader == null) {
+      // Schema ghost corner case!  This field did index points in the past, but
+      // now all docs having this field were deleted in this segment:
+      return 0L;
+    }
+    return bkdReader.estimatePointCount(visitor);
+  }
+
+  @Override
   public void checkIntegrity() throws IOException {
     BytesRefBuilder scratch = new BytesRefBuilder();
     IndexInput clone = dataIn.clone();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/java/org/apache/lucene/codecs/PointsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/PointsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/PointsFormat.java
index e49bf53..968c505 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/PointsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/PointsFormat.java
@@ -77,6 +77,11 @@ public abstract class PointsFormat {
           }
 
           @Override
+          public long estimatePointCount(String fieldName, IntersectVisitor visitor) {
+            throw new IllegalArgumentException("field=\"" + fieldName + "\" was not indexed with points");
+          }
+
+          @Override
           public byte[] getMinPackedValue(String fieldName) {
             throw new IllegalArgumentException("field=\"" + fieldName + "\" was not indexed with points");
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
index eac1716..de3ede6 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
@@ -122,6 +122,11 @@ public abstract class PointsWriter implements Closeable {
                  }
 
                  @Override
+                 public long estimatePointCount(String fieldName, IntersectVisitor visitor) {
+                   throw new UnsupportedOperationException();
+                 }
+
+                 @Override
                  public byte[] getMinPackedValue(String fieldName) {
                    throw new UnsupportedOperationException();
                  }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsReader.java
index 8c91a99..fe7a7c0 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsReader.java
@@ -143,6 +143,19 @@ public class Lucene60PointsReader extends PointsReader implements Closeable {
   }
 
   @Override
+  public long estimatePointCount(String fieldName, IntersectVisitor visitor) {
+    BKDReader bkdReader = getBKDReader(fieldName);
+
+    if (bkdReader == null) {
+      // Schema ghost corner case!  This field did index points in the past, but
+      // now all docs having this point field were deleted in this segment:
+      return 0;
+    }
+
+    return bkdReader.estimatePointCount(visitor);
+  }
+
+  @Override
   public long ramBytesUsed() {
     long sizeInBytes = 0;
     for(BKDReader reader : readers.values()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
index 21a28e5..27e878b 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
@@ -42,6 +42,8 @@ import org.apache.lucene.codecs.TermVectorsReader;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DocumentStoredFieldVisitor;
 import org.apache.lucene.index.CheckIndex.Status.DocValuesStatus;
+import org.apache.lucene.index.PointValues.IntersectVisitor;
+import org.apache.lucene.index.PointValues.Relation;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.LeafFieldComparator;
 import org.apache.lucene.search.Sort;
@@ -1805,6 +1807,19 @@ public final class CheckIndex implements Closeable {
             long size = values.size(fieldInfo.name);
             int docCount = values.getDocCount(fieldInfo.name);
 
+            final long crossCost = values.estimatePointCount(fieldInfo.name, new ConstantRelationIntersectVisitor(Relation.CELL_CROSSES_QUERY));
+            if (crossCost < size) {
+              throw new RuntimeException("estimatePointCount should return >= size when all cells match");
+            }
+            final long insideCost = values.estimatePointCount(fieldInfo.name, new ConstantRelationIntersectVisitor(Relation.CELL_INSIDE_QUERY));
+            if (insideCost < size) {
+              throw new RuntimeException("estimatePointCount should return >= size when all cells fully match");
+            }
+            final long outsideCost = values.estimatePointCount(fieldInfo.name, new ConstantRelationIntersectVisitor(Relation.CELL_OUTSIDE_QUERY));
+            if (outsideCost != 0) {
+              throw new RuntimeException("estimatePointCount should return 0 when no cells match");
+            }
+
             VerifyPointsVisitor visitor = new VerifyPointsVisitor(fieldInfo.name, reader.maxDoc(), values);
             values.intersect(fieldInfo.name, visitor);
 
@@ -1997,6 +2012,28 @@ public final class CheckIndex implements Closeable {
     }
   }
 
+  private static class ConstantRelationIntersectVisitor implements IntersectVisitor {
+    private final Relation relation;
+
+    ConstantRelationIntersectVisitor(Relation relation) {
+      this.relation = relation;
+    }
+
+    @Override
+    public void visit(int docID) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void visit(int docID, byte[] packedValue) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+      return relation;
+    }
+  }
   
   /**
    * Test stored fields.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java b/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java
index d85ff2d..1b9ab48 100644
--- a/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java
@@ -344,6 +344,19 @@ public class ParallelLeafReader extends LeafReader {
       }
 
       @Override
+      public long estimatePointCount(String fieldName, IntersectVisitor visitor) {
+        LeafReader reader = fieldToReader.get(fieldName);
+        if (reader == null) {
+          return 0;
+        }
+        PointValues dimValues = reader.getPointValues();
+        if (dimValues == null) {
+          return 0;
+        }
+        return dimValues.estimatePointCount(fieldName, visitor);
+      }
+
+      @Override
       public byte[] getMinPackedValue(String fieldName) throws IOException {
         LeafReader reader = fieldToReader.get(fieldName);
         if (reader == null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/java/org/apache/lucene/index/PointValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PointValues.java b/lucene/core/src/java/org/apache/lucene/index/PointValues.java
index 528f971..a4901a2 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PointValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PointValues.java
@@ -26,6 +26,7 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FloatPoint;
 import org.apache.lucene.document.IntPoint;
 import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.bkd.BKDWriter;
 
@@ -226,6 +227,12 @@ public abstract class PointValues {
    *  to test whether each document is deleted, if necessary. */
   public abstract void intersect(String fieldName, IntersectVisitor visitor) throws IOException;
 
+  /** Estimate the number of points that would be visited by {@link #intersect}
+   * with the given {@link IntersectVisitor}. This should run many times faster
+   * than {@link #intersect(String, IntersectVisitor)}.
+   * @see DocIdSetIterator#cost */
+  public abstract long estimatePointCount(String fieldName, IntersectVisitor visitor);
+
   /** Returns minimum value for each dimension, packed, or null if {@link #size} is <code>0</code> */
   public abstract byte[] getMinPackedValue(String fieldName) throws IOException;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
index cc14bd2..3d41d6d 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
@@ -108,6 +108,11 @@ class PointValuesWriter {
       }
 
       @Override
+      public long estimatePointCount(String fieldName, IntersectVisitor visitor) {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
       public byte[] getMinPackedValue(String fieldName) {
         throw new UnsupportedOperationException();
       }
@@ -274,4 +279,5 @@ class PointValuesWriter {
       return in.ramBytesUsed();
     }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/java/org/apache/lucene/index/SlowCodecReaderWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SlowCodecReaderWrapper.java b/lucene/core/src/java/org/apache/lucene/index/SlowCodecReaderWrapper.java
index 2742247..8e06d32 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SlowCodecReaderWrapper.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SlowCodecReaderWrapper.java
@@ -151,6 +151,11 @@ public final class SlowCodecReaderWrapper {
       }
 
       @Override
+      public long estimatePointCount(String fieldName, IntersectVisitor visitor) {
+        return values.estimatePointCount(fieldName, visitor);
+      }
+
+      @Override
       public void checkIntegrity() throws IOException {
         // We already checkIntegrity the entire reader up front
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java b/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
index 1b57594..4ea3541 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
@@ -259,6 +259,11 @@ class SortingLeafReader extends FilterLeafReader {
     }
 
     @Override
+    public long estimatePointCount(String fieldName, IntersectVisitor visitor) {
+      return in.estimatePointCount(fieldName, visitor);
+    }
+
+    @Override
     public byte[] getMinPackedValue(String fieldName) throws IOException {
       return in.getMinPackedValue(fieldName);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java b/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java
new file mode 100644
index 0000000..8dfbe98
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java
@@ -0,0 +1,265 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.OptionalLong;
+import java.util.stream.Stream;
+
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.util.PriorityQueue;
+
+final class Boolean2ScorerSupplier extends ScorerSupplier {
+
+  private final BooleanWeight weight;
+  private final Map<BooleanClause.Occur, Collection<ScorerSupplier>> subs;
+  private final boolean needsScores;
+  private final boolean disableCoord;
+  private final float[] coords;
+  private final int maxCoord;
+  private final int minShouldMatch;
+  private long cost = -1;
+
+  Boolean2ScorerSupplier(BooleanWeight weight,
+      Map<Occur, Collection<ScorerSupplier>> subs,
+      boolean disableCoord, float[] coords, int maxCoord,
+      boolean needsScores, int minShouldMatch) {
+    if (minShouldMatch < 0) {
+      throw new IllegalArgumentException("minShouldMatch must be positive, but got: " + minShouldMatch);
+    }
+    if (minShouldMatch != 0 && minShouldMatch >= subs.get(Occur.SHOULD).size()) {
+      throw new IllegalArgumentException("minShouldMatch must be strictly less than the number of SHOULD clauses");
+    }
+    if (needsScores == false && minShouldMatch == 0 && subs.get(Occur.SHOULD).size() > 0
+        && subs.get(Occur.MUST).size() + subs.get(Occur.FILTER).size() > 0) {
+      throw new IllegalArgumentException("Cannot pass purely optional clauses if scores are not needed");
+    }
+    if (subs.get(Occur.SHOULD).size() + subs.get(Occur.MUST).size() + subs.get(Occur.FILTER).size() == 0) {
+      throw new IllegalArgumentException("There should be at least one positive clause");
+    }
+    this.weight = weight;
+    this.subs = subs;
+    this.disableCoord = disableCoord;
+    this.coords = coords;
+    this.maxCoord = maxCoord;
+    this.needsScores = needsScores;
+    this.minShouldMatch = minShouldMatch;
+  }
+
+  private long computeCost() {
+    OptionalLong minRequiredCost = Stream.concat(
+        subs.get(Occur.MUST).stream(),
+        subs.get(Occur.FILTER).stream())
+        .mapToLong(ScorerSupplier::cost)
+        .min();
+    if (minRequiredCost.isPresent() && minShouldMatch == 0) {
+      return minRequiredCost.getAsLong();
+    } else {
+      final Collection<ScorerSupplier> optionalScorers = subs.get(Occur.SHOULD);
+      final long shouldCost = MinShouldMatchSumScorer.cost(
+          optionalScorers.stream().mapToLong(ScorerSupplier::cost),
+          optionalScorers.size(), minShouldMatch);
+      return Math.min(minRequiredCost.orElse(Long.MAX_VALUE), shouldCost);
+    }
+  }
+
+  @Override
+  public long cost() {
+    if (cost == -1) {
+      cost = computeCost();
+    }
+    return cost;
+  }
+
+  @Override
+  public Scorer get(boolean randomAccess) throws IOException {
+    // three cases: conjunction, disjunction, or mix
+
+    // pure conjunction
+    if (subs.get(Occur.SHOULD).isEmpty()) {
+      return excl(req(subs.get(Occur.FILTER), subs.get(Occur.MUST), disableCoord, randomAccess), subs.get(Occur.MUST_NOT));
+    }
+
+    // pure disjunction
+    if (subs.get(Occur.FILTER).isEmpty() && subs.get(Occur.MUST).isEmpty()) {
+      return excl(opt(subs.get(Occur.SHOULD), minShouldMatch, needsScores, disableCoord, randomAccess), subs.get(Occur.MUST_NOT));
+    }
+
+    // conjunction-disjunction mix:
+    // we create the required and optional pieces, and then
+    // combine the two: if minNrShouldMatch > 0, then it's a conjunction: because the
+    // optional side must match. otherwise it's required + optional
+
+    boolean reqRandomAccess = randomAccess;
+    boolean shouldRandomAccess = randomAccess || minShouldMatch == 0;
+    if (randomAccess == false && minShouldMatch > 0) {
+      // We need to figure out whether the MUST/FILTER or the SHOULD clauses would lead the iteration
+      final long reqCost = Stream.concat(
+          subs.get(Occur.MUST).stream(),
+          subs.get(Occur.FILTER).stream())
+          .mapToLong(ScorerSupplier::cost)
+          .min().getAsLong();
+      final long msmCost = MinShouldMatchSumScorer.cost(
+          subs.get(Occur.SHOULD).stream().mapToLong(ScorerSupplier::cost),
+          subs.get(Occur.SHOULD).size(), minShouldMatch);
+      reqRandomAccess = reqCost > msmCost;
+      shouldRandomAccess = msmCost > reqCost;
+    }
+
+    Scorer req = excl(req(subs.get(Occur.FILTER), subs.get(Occur.MUST), true, reqRandomAccess), subs.get(Occur.MUST_NOT));
+    Scorer opt = opt(subs.get(Occur.SHOULD), minShouldMatch, needsScores, true, shouldRandomAccess);
+
+    
+    // TODO: clean this up: it's horrible
+    if (disableCoord) {
+      if (minShouldMatch > 0) {
+        return new ConjunctionScorer(weight, Arrays.asList(req, opt), Arrays.asList(req, opt), 1F);
+      } else {
+        return new ReqOptSumScorer(req, opt);          
+      }
+    } else if (subs.get(Occur.SHOULD).size() == 1) {
+      if (minShouldMatch > 0) {
+        return new ConjunctionScorer(weight, Arrays.asList(req, opt), Arrays.asList(req, opt), weight.coord(subs.get(Occur.MUST).size()+1, maxCoord));
+      } else {
+        float coordReq = weight.coord(subs.get(Occur.MUST).size(), maxCoord);
+        float coordBoth = weight.coord(subs.get(Occur.MUST).size() + 1, maxCoord);
+        return new BooleanTopLevelScorers.ReqSingleOptScorer(req, opt, coordReq, coordBoth);
+      }
+    } else {
+      if (minShouldMatch > 0) {
+        return new BooleanTopLevelScorers.CoordinatingConjunctionScorer(weight, coords, req, subs.get(Occur.MUST).size(), opt);
+      } else {
+        return new BooleanTopLevelScorers.ReqMultiOptScorer(req, opt, subs.get(Occur.MUST).size(), coords); 
+      }
+    }
+  }
+
+  /** Create a new scorer for the given required clauses. Note that
+   *  {@code requiredScoring} is a subset of {@code required} containing
+   *  required clauses that should participate in scoring. */
+  private Scorer req(Collection<ScorerSupplier> requiredNoScoring, Collection<ScorerSupplier> requiredScoring,
+      boolean disableCoord, boolean randomAccess) throws IOException {
+    if (requiredNoScoring.size() + requiredScoring.size() == 1) {
+      Scorer req = (requiredNoScoring.isEmpty() ? requiredScoring : requiredNoScoring).iterator().next().get(randomAccess);
+
+      if (needsScores == false) {
+        return req;
+      }
+
+      if (requiredScoring.isEmpty()) {
+        // Scores are needed but we only have a filter clause
+        // BooleanWeight expects that calling score() is ok so we need to wrap
+        // to prevent score() from being propagated
+        return new FilterScorer(req) {
+          @Override
+          public float score() throws IOException {
+            return 0f;
+          }
+          @Override
+          public int freq() throws IOException {
+            return 0;
+          }
+        };
+      }
+
+      float boost = 1f;
+      if (disableCoord == false) {
+        boost = weight.coord(1, maxCoord);
+      }
+      if (boost == 1f) {
+        return req;
+      }
+      return new BooleanTopLevelScorers.BoostedScorer(req, boost);
+    } else {
+      long minCost = Math.min(
+          requiredNoScoring.stream().mapToLong(ScorerSupplier::cost).min().orElse(Long.MAX_VALUE),
+          requiredScoring.stream().mapToLong(ScorerSupplier::cost).min().orElse(Long.MAX_VALUE));
+      List<Scorer> requiredScorers = new ArrayList<>();
+      List<Scorer> scoringScorers = new ArrayList<>();
+      for (ScorerSupplier s : requiredNoScoring) {
+        requiredScorers.add(s.get(randomAccess || s.cost() > minCost));
+      }
+      for (ScorerSupplier s : requiredScoring) {
+        Scorer scorer = s.get(randomAccess || s.cost() > minCost);
+        requiredScorers.add(scorer);
+        scoringScorers.add(scorer);
+      }
+      return new ConjunctionScorer(weight, requiredScorers, scoringScorers,
+          disableCoord || needsScores == false ? 1.0F : weight.coord(requiredScoring.size(), maxCoord));
+    }
+  }
+
+  private Scorer excl(Scorer main, Collection<ScorerSupplier> prohibited) throws IOException {
+    if (prohibited.isEmpty()) {
+      return main;
+    } else {
+      return new ReqExclScorer(main, opt(prohibited, 1, false, true, true));
+    }
+  }
+
+  private Scorer opt(Collection<ScorerSupplier> optional, int minShouldMatch,
+      boolean needsScores, boolean disableCoord, boolean randomAccess) throws IOException {
+    if (optional.size() == 1) {
+      Scorer opt = optional.iterator().next().get(randomAccess);
+      if (needsScores && !disableCoord && maxCoord > 1) {
+        return new BooleanTopLevelScorers.BoostedScorer(opt, weight.coord(1, maxCoord));
+      } else {
+        return opt;
+      }
+    } else {
+      float coords[];
+      if (disableCoord || needsScores == false) {
+        // sneaky: when we do a mixed conjunction/disjunction, we need a fake for the disjunction part.
+        coords = new float[optional.size()+1];
+        Arrays.fill(coords, 1F);
+      } else {
+        coords = this.coords;
+      }
+      if (minShouldMatch > 1) {
+        final List<Scorer> optionalScorers = new ArrayList<>();
+        final PriorityQueue<ScorerSupplier> pq = new PriorityQueue<ScorerSupplier>(subs.get(Occur.SHOULD).size() - minShouldMatch + 1) {
+          @Override
+          protected boolean lessThan(ScorerSupplier a, ScorerSupplier b) {
+            return a.cost() > b.cost();
+          }
+        };
+        for (ScorerSupplier scorer : subs.get(Occur.SHOULD)) {
+          ScorerSupplier overflow = pq.insertWithOverflow(scorer);
+          if (overflow != null) {
+            optionalScorers.add(overflow.get(true));
+          }
+        }
+        for (ScorerSupplier scorer : pq) {
+          optionalScorers.add(scorer.get(randomAccess));
+        }
+        return new MinShouldMatchSumScorer(weight, optionalScorers, minShouldMatch, coords);
+      } else {
+        final List<Scorer> optionalScorers = new ArrayList<>();
+        for (ScorerSupplier scorer : optional) {
+          optionalScorers.add(scorer.get(randomAccess));
+        }
+        return new DisjunctionSumScorer(weight, optionalScorers, coords, needsScores);
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/java/org/apache/lucene/search/BooleanTopLevelScorers.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanTopLevelScorers.java b/lucene/core/src/java/org/apache/lucene/search/BooleanTopLevelScorers.java
index d34e708..d9d68b6 100644
--- a/lucene/core/src/java/org/apache/lucene/search/BooleanTopLevelScorers.java
+++ b/lucene/core/src/java/org/apache/lucene/search/BooleanTopLevelScorers.java
@@ -95,7 +95,7 @@ class BooleanTopLevelScorers {
     private final Scorer opt;
     
     CoordinatingConjunctionScorer(Weight weight, float coords[], Scorer req, int reqCount, Scorer opt) {
-      super(weight, Arrays.asList(req, opt), Arrays.asList(req, opt));
+      super(weight, Arrays.asList(req, opt), Arrays.asList(req, opt), 1f);
       this.coords = coords;
       this.req = req;
       this.reqCount = reqCount;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
index 24cbac8..12a710b 100644
--- a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
+++ b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
@@ -20,8 +20,11 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
+import java.util.EnumMap;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.lucene.index.LeafReaderContext;
@@ -350,7 +353,9 @@ final class BooleanWeight extends Weight {
     if (prohibited.isEmpty()) {
       return positiveScorer;
     } else {
-      Scorer prohibitedScorer = opt(prohibited, 1, true);
+      Scorer prohibitedScorer = prohibited.size() == 1
+          ? prohibited.get(0)
+          : new DisjunctionSumScorer(this, prohibited, coords, false);
       if (prohibitedScorer.twoPhaseIterator() != null) {
         // ReqExclBulkScorer can't deal efficiently with two-phased prohibited clauses
         return null;
@@ -373,50 +378,48 @@ final class BooleanWeight extends Weight {
 
   @Override
   public Scorer scorer(LeafReaderContext context) throws IOException {
-    // initially the user provided value,
-    // but if minNrShouldMatch == optional.size(),
-    // we will optimize and move these to required, making this 0
+    ScorerSupplier scorerSupplier = scorerSupplier(context);
+    if (scorerSupplier == null) {
+      return null;
+    }
+    return scorerSupplier.get(false);
+  }
+
+  @Override
+  public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
     int minShouldMatch = query.getMinimumNumberShouldMatch();
 
-    List<Scorer> required = new ArrayList<>();
-    // clauses that are required AND participate in scoring, subset of 'required'
-    List<Scorer> requiredScoring = new ArrayList<>();
-    List<Scorer> prohibited = new ArrayList<>();
-    List<Scorer> optional = new ArrayList<>();
+    final Map<Occur, Collection<ScorerSupplier>> scorers = new EnumMap<>(Occur.class);
+    for (Occur occur : Occur.values()) {
+      scorers.put(occur, new ArrayList<>());
+    }
+
     Iterator<BooleanClause> cIter = query.iterator();
     for (Weight w  : weights) {
       BooleanClause c =  cIter.next();
-      Scorer subScorer = w.scorer(context);
+      ScorerSupplier subScorer = w.scorerSupplier(context);
       if (subScorer == null) {
         if (c.isRequired()) {
           return null;
         }
-      } else if (c.isRequired()) {
-        required.add(subScorer);
-        if (c.isScoring()) {
-          requiredScoring.add(subScorer);
-        }
-      } else if (c.isProhibited()) {
-        prohibited.add(subScorer);
       } else {
-        optional.add(subScorer);
+        scorers.get(c.getOccur()).add(subScorer);
       }
     }
-    
+
     // scorer simplifications:
     
-    if (optional.size() == minShouldMatch) {
+    if (scorers.get(Occur.SHOULD).size() == minShouldMatch) {
       // any optional clauses are in fact required
-      required.addAll(optional);
-      requiredScoring.addAll(optional);
-      optional.clear();
+      scorers.get(Occur.MUST).addAll(scorers.get(Occur.SHOULD));
+      scorers.get(Occur.SHOULD).clear();
       minShouldMatch = 0;
     }
     
-    if (required.isEmpty() && optional.isEmpty()) {
+    if (scorers.get(Occur.FILTER).isEmpty() && scorers.get(Occur.MUST).isEmpty() && scorers.get(Occur.SHOULD).isEmpty()) {
       // no required and optional clauses.
       return null;
-    } else if (optional.size() < minShouldMatch) {
+    } else if (scorers.get(Occur.SHOULD).size() < minShouldMatch) {
       // either >1 req scorer, or there are 0 req scorers and at least 1
       // optional scorer. Therefore if there are not enough optional scorers
       // no documents will be matched by the query
@@ -424,130 +427,10 @@ final class BooleanWeight extends Weight {
     }
 
     // we don't need scores, so if we have required clauses, drop optional clauses completely
-    if (!needsScores && minShouldMatch == 0 && required.size() > 0) {
-      optional.clear();
+    if (!needsScores && minShouldMatch == 0 && scorers.get(Occur.MUST).size() + scorers.get(Occur.FILTER).size() > 0) {
+      scorers.get(Occur.SHOULD).clear();
     }
-    
-    // three cases: conjunction, disjunction, or mix
-    
-    // pure conjunction
-    if (optional.isEmpty()) {
-      return excl(req(required, requiredScoring, disableCoord), prohibited);
-    }
-    
-    // pure disjunction
-    if (required.isEmpty()) {
-      return excl(opt(optional, minShouldMatch, disableCoord), prohibited);
-    }
-    
-    // conjunction-disjunction mix:
-    // we create the required and optional pieces with coord disabled, and then
-    // combine the two: if minNrShouldMatch > 0, then it's a conjunction: because the
-    // optional side must match. otherwise it's required + optional, factoring the
-    // number of optional terms into the coord calculation
-    
-    Scorer req = excl(req(required, requiredScoring, true), prohibited);
-    Scorer opt = opt(optional, minShouldMatch, true);
-
-    // TODO: clean this up: it's horrible
-    if (disableCoord) {
-      if (minShouldMatch > 0) {
-        return new ConjunctionScorer(this, Arrays.asList(req, opt), Arrays.asList(req, opt), 1F);
-      } else {
-        return new ReqOptSumScorer(req, opt);          
-      }
-    } else if (optional.size() == 1) {
-      if (minShouldMatch > 0) {
-        return new ConjunctionScorer(this, Arrays.asList(req, opt), Arrays.asList(req, opt), coord(requiredScoring.size()+1, maxCoord));
-      } else {
-        float coordReq = coord(requiredScoring.size(), maxCoord);
-        float coordBoth = coord(requiredScoring.size() + 1, maxCoord);
-        return new BooleanTopLevelScorers.ReqSingleOptScorer(req, opt, coordReq, coordBoth);
-      }
-    } else {
-      if (minShouldMatch > 0) {
-        return new BooleanTopLevelScorers.CoordinatingConjunctionScorer(this, coords, req, requiredScoring.size(), opt);
-      } else {
-        return new BooleanTopLevelScorers.ReqMultiOptScorer(req, opt, requiredScoring.size(), coords); 
-      }
-    }
-  }
 
-  /** Create a new scorer for the given required clauses. Note that
-   *  {@code requiredScoring} is a subset of {@code required} containing
-   *  required clauses that should participate in scoring. */
-  private Scorer req(List<Scorer> required, List<Scorer> requiredScoring, boolean disableCoord) {
-    if (required.size() == 1) {
-      Scorer req = required.get(0);
-
-      if (needsScores == false) {
-        return req;
-      }
-
-      if (requiredScoring.isEmpty()) {
-        // Scores are needed but we only have a filter clause
-        // BooleanWeight expects that calling score() is ok so we need to wrap
-        // to prevent score() from being propagated
-        return new FilterScorer(req) {
-          @Override
-          public float score() throws IOException {
-            return 0f;
-          }
-          @Override
-          public int freq() throws IOException {
-            return 0;
-          }
-        };
-      }
-      
-      float boost = 1f;
-      if (disableCoord == false) {
-        boost = coord(1, maxCoord);
-      }
-      if (boost == 1f) {
-        return req;
-      }
-      return new BooleanTopLevelScorers.BoostedScorer(req, boost);
-    } else {
-      return new ConjunctionScorer(this, required, requiredScoring,
-                                   disableCoord ? 1.0F : coord(requiredScoring.size(), maxCoord));
-    }
-  }
-  
-  private Scorer excl(Scorer main, List<Scorer> prohibited) throws IOException {
-    if (prohibited.isEmpty()) {
-      return main;
-    } else if (prohibited.size() == 1) {
-      return new ReqExclScorer(main, prohibited.get(0));
-    } else {
-      float coords[] = new float[prohibited.size()+1];
-      Arrays.fill(coords, 1F);
-      return new ReqExclScorer(main, new DisjunctionSumScorer(this, prohibited, coords, false));
-    }
-  }
-  
-  private Scorer opt(List<Scorer> optional, int minShouldMatch, boolean disableCoord) throws IOException {
-    if (optional.size() == 1) {
-      Scorer opt = optional.get(0);
-      if (!disableCoord && maxCoord > 1) {
-        return new BooleanTopLevelScorers.BoostedScorer(opt, coord(1, maxCoord));
-      } else {
-        return opt;
-      }
-    } else {
-      float coords[];
-      if (disableCoord) {
-        // sneaky: when we do a mixed conjunction/disjunction, we need a fake for the disjunction part.
-        coords = new float[optional.size()+1];
-        Arrays.fill(coords, 1F);
-      } else {
-        coords = this.coords;
-      }
-      if (minShouldMatch > 1) {
-        return new MinShouldMatchSumScorer(this, optional, minShouldMatch, coords);
-      } else {
-        return new DisjunctionSumScorer(this, optional, coords, needsScores);
-      }
-    }
+    return new Boolean2ScorerSupplier(this, scorers, disableCoord, coords, maxCoord, needsScores, minShouldMatch);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/java/org/apache/lucene/search/ConjunctionDISI.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/ConjunctionDISI.java b/lucene/core/src/java/org/apache/lucene/search/ConjunctionDISI.java
index 43d03b2..780e854 100644
--- a/lucene/core/src/java/org/apache/lucene/search/ConjunctionDISI.java
+++ b/lucene/core/src/java/org/apache/lucene/search/ConjunctionDISI.java
@@ -41,7 +41,7 @@ public final class ConjunctionDISI extends DocIdSetIterator {
    * returned {@link DocIdSetIterator} might leverage two-phase iteration in
    * which case it is possible to retrieve the {@link TwoPhaseIterator} using
    * {@link TwoPhaseIterator#unwrap}. */
-  public static DocIdSetIterator intersectScorers(List<Scorer> scorers) {
+  public static DocIdSetIterator intersectScorers(Collection<Scorer> scorers) {
     if (scorers.size() < 2) {
       throw new IllegalArgumentException("Cannot make a ConjunctionDISI of less than 2 iterators");
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java b/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
index ab1de92..f2074f3 100644
--- a/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
@@ -20,7 +20,6 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.List;
 
 /** Scorer for conjunctions, sets of queries, all of which are required. */
 class ConjunctionScorer extends Scorer {
@@ -29,12 +28,8 @@ class ConjunctionScorer extends Scorer {
   final Scorer[] scorers;
   final float coord;
 
-  ConjunctionScorer(Weight weight, List<Scorer> required, List<Scorer> scorers) {
-    this(weight, required, scorers, 1f);
-  }
-
   /** Create a new {@link ConjunctionScorer}, note that {@code scorers} must be a subset of {@code required}. */
-  ConjunctionScorer(Weight weight, List<Scorer> required, List<Scorer> scorers, float coord) {
+  ConjunctionScorer(Weight weight, Collection<Scorer> required, Collection<Scorer> scorers, float coord) {
     super(weight);
     assert required.containsAll(scorers);
     this.coord = coord;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java b/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
index eb5e2d3..772e431 100644
--- a/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
@@ -125,28 +125,48 @@ public final class ConstantScoreQuery extends Query {
         }
 
         @Override
-        public Scorer scorer(LeafReaderContext context) throws IOException {
-          final Scorer innerScorer = innerWeight.scorer(context);
-          if (innerScorer == null) {
+        public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
+          ScorerSupplier innerScorerSupplier = innerWeight.scorerSupplier(context);
+          if (innerScorerSupplier == null) {
             return null;
           }
-          final float score = score();
-          return new FilterScorer(innerScorer) {
+          return new ScorerSupplier() {
             @Override
-            public float score() throws IOException {
-              return score;
-            }
-            @Override
-            public int freq() throws IOException {
-              return 1;
+            public Scorer get(boolean randomAccess) throws IOException {
+              final Scorer innerScorer = innerScorerSupplier.get(randomAccess);
+              final float score = score();
+              return new FilterScorer(innerScorer) {
+                @Override
+                public float score() throws IOException {
+                  return score;
+                }
+                @Override
+                public int freq() throws IOException {
+                  return 1;
+                }
+                @Override
+                public Collection<ChildScorer> getChildren() {
+                  return Collections.singleton(new ChildScorer(innerScorer, "constant"));
+                }
+              };
             }
+
             @Override
-            public Collection<ChildScorer> getChildren() {
-              return Collections.singleton(new ChildScorer(innerScorer, "constant"));
+            public long cost() {
+              return innerScorerSupplier.cost();
             }
           };
         }
 
+        @Override
+        public Scorer scorer(LeafReaderContext context) throws IOException {
+          ScorerSupplier scorerSupplier = scorerSupplier(context);
+          if (scorerSupplier == null) {
+            return null;
+          }
+          return scorerSupplier.get(false);
+        }
+
       };
     } else {
       return innerWeight;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java b/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
index 9653d95..0a0f013 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
@@ -22,6 +22,8 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.stream.LongStream;
+import java.util.stream.StreamSupport;
 
 import org.apache.lucene.util.PriorityQueue;
 
@@ -47,7 +49,7 @@ import static org.apache.lucene.search.DisiPriorityQueue.rightNode;
  */
 final class MinShouldMatchSumScorer extends Scorer {
 
-  private static long cost(Collection<Scorer> scorers, int minShouldMatch) {
+  static long cost(LongStream costs, int numScorers, int minShouldMatch) {
     // the idea here is the following: a boolean query c1,c2,...cn with minShouldMatch=m
     // could be rewritten to:
     // (c1 AND (c2..cn|msm=m-1)) OR (!c1 AND (c2..cn|msm=m))
@@ -61,20 +63,14 @@ final class MinShouldMatchSumScorer extends Scorer {
 
     // If we recurse infinitely, we find out that the cost of a msm query is the sum of the
     // costs of the num_scorers - minShouldMatch + 1 least costly scorers
-    final PriorityQueue<Scorer> pq = new PriorityQueue<Scorer>(scorers.size() - minShouldMatch + 1) {
+    final PriorityQueue<Long> pq = new PriorityQueue<Long>(numScorers - minShouldMatch + 1) {
       @Override
-      protected boolean lessThan(Scorer a, Scorer b) {
-        return a.iterator().cost() > b.iterator().cost();
+      protected boolean lessThan(Long a, Long b) {
+        return a > b;
       }
     };
-    for (Scorer scorer : scorers) {
-      pq.insertWithOverflow(scorer);
-    }
-    long cost = 0;
-    for (Scorer scorer = pq.pop(); scorer != null; scorer = pq.pop()) {
-      cost += scorer.iterator().cost();
-    }
-    return cost;
+    costs.forEach(pq::insertWithOverflow);
+    return StreamSupport.stream(pq.spliterator(), false).mapToLong(Number::longValue).sum();
   }
 
   final int minShouldMatch;
@@ -126,7 +122,7 @@ final class MinShouldMatchSumScorer extends Scorer {
       children.add(new ChildScorer(scorer, "SHOULD"));
     }
     this.childScorers = Collections.unmodifiableCollection(children);
-    this.cost = cost(scorers, minShouldMatch);
+    this.cost = cost(scorers.stream().map(Scorer::iterator).mapToLong(DocIdSetIterator::cost), scorers.size(), minShouldMatch);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java
index 63de04c..20e9f33 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java
@@ -105,71 +105,67 @@ public abstract class PointRangeQuery extends Query {
 
     return new ConstantScoreWeight(this) {
 
-      private DocIdSet buildMatchingDocIdSet(LeafReader reader, PointValues values) throws IOException {
-        DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
+      private IntersectVisitor getIntersectVisitor(DocIdSetBuilder result) {
+        return new IntersectVisitor() {
 
-        values.intersect(field,
-            new IntersectVisitor() {
+          DocIdSetBuilder.BulkAdder adder;
 
-              DocIdSetBuilder.BulkAdder adder;
+          @Override
+          public void grow(int count) {
+            adder = result.grow(count);
+          }
 
-              @Override
-              public void grow(int count) {
-                adder = result.grow(count);
-              }
+          @Override
+          public void visit(int docID) {
+            adder.add(docID);
+          }
 
-              @Override
-              public void visit(int docID) {
-                adder.add(docID);
+          @Override
+          public void visit(int docID, byte[] packedValue) {
+            for(int dim=0;dim<numDims;dim++) {
+              int offset = dim*bytesPerDim;
+              if (StringHelper.compare(bytesPerDim, packedValue, offset, lowerPoint, offset) < 0) {
+                // Doc's value is too low, in this dimension
+                return;
               }
-
-              @Override
-              public void visit(int docID, byte[] packedValue) {
-                for(int dim=0;dim<numDims;dim++) {
-                  int offset = dim*bytesPerDim;
-                  if (StringHelper.compare(bytesPerDim, packedValue, offset, lowerPoint, offset) < 0) {
-                    // Doc's value is too low, in this dimension
-                    return;
-                  }
-                  if (StringHelper.compare(bytesPerDim, packedValue, offset, upperPoint, offset) > 0) {
-                    // Doc's value is too high, in this dimension
-                    return;
-                  }
-                }
-
-                // Doc is in-bounds
-                adder.add(docID);
+              if (StringHelper.compare(bytesPerDim, packedValue, offset, upperPoint, offset) > 0) {
+                // Doc's value is too high, in this dimension
+                return;
               }
+            }
 
-              @Override
-              public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-
-                boolean crosses = false;
+            // Doc is in-bounds
+            adder.add(docID);
+          }
 
-                for(int dim=0;dim<numDims;dim++) {
-                  int offset = dim*bytesPerDim;
+          @Override
+          public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
 
-                  if (StringHelper.compare(bytesPerDim, minPackedValue, offset, upperPoint, offset) > 0 ||
-                      StringHelper.compare(bytesPerDim, maxPackedValue, offset, lowerPoint, offset) < 0) {
-                    return Relation.CELL_OUTSIDE_QUERY;
-                  }
+            boolean crosses = false;
 
-                  crosses |= StringHelper.compare(bytesPerDim, minPackedValue, offset, lowerPoint, offset) < 0 ||
-                    StringHelper.compare(bytesPerDim, maxPackedValue, offset, upperPoint, offset) > 0;
-                }
+            for(int dim=0;dim<numDims;dim++) {
+              int offset = dim*bytesPerDim;
 
-                if (crosses) {
-                  return Relation.CELL_CROSSES_QUERY;
-                } else {
-                  return Relation.CELL_INSIDE_QUERY;
-                }
+              if (StringHelper.compare(bytesPerDim, minPackedValue, offset, upperPoint, offset) > 0 ||
+                  StringHelper.compare(bytesPerDim, maxPackedValue, offset, lowerPoint, offset) < 0) {
+                return Relation.CELL_OUTSIDE_QUERY;
               }
-            });
-        return result.build();
+
+              crosses |= StringHelper.compare(bytesPerDim, minPackedValue, offset, lowerPoint, offset) < 0 ||
+                  StringHelper.compare(bytesPerDim, maxPackedValue, offset, upperPoint, offset) > 0;
+            }
+
+            if (crosses) {
+              return Relation.CELL_CROSSES_QUERY;
+            } else {
+              return Relation.CELL_INSIDE_QUERY;
+            }
+          }
+        };
       }
 
       @Override
-      public Scorer scorer(LeafReaderContext context) throws IOException {
+      public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
         LeafReader reader = context.reader();
         PointValues values = reader.getPointValues();
         if (values == null) {
@@ -205,15 +201,55 @@ public abstract class PointRangeQuery extends Query {
           allDocsMatch = false;
         }
 
-        DocIdSetIterator iterator;
+        final Weight weight = this;
         if (allDocsMatch) {
           // all docs have a value and all points are within bounds, so everything matches
-          iterator = DocIdSetIterator.all(reader.maxDoc());
+          return new ScorerSupplier() {
+            @Override
+            public Scorer get(boolean randomAccess) {
+              return new ConstantScoreScorer(weight, score(),
+                  DocIdSetIterator.all(reader.maxDoc()));
+            }
+            
+            @Override
+            public long cost() {
+              return reader.maxDoc();
+            }
+          };
         } else {
-          iterator = buildMatchingDocIdSet(reader, values).iterator();
+          return new ScorerSupplier() {
+
+            final DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
+            final IntersectVisitor visitor = getIntersectVisitor(result);
+            long cost = -1;
+
+            @Override
+            public Scorer get(boolean randomAccess) throws IOException {
+              values.intersect(field, visitor);
+              DocIdSetIterator iterator = result.build().iterator();
+              return new ConstantScoreScorer(weight, score(), iterator);
+            }
+            
+            @Override
+            public long cost() {
+              if (cost == -1) {
+                // Computing the cost may be expensive, so only do it if necessary
+                cost = values.estimatePointCount(field, visitor);
+                assert cost >= 0;
+              }
+              return cost;
+            }
+          };
         }
+      }
 
-        return new ConstantScoreScorer(this, score(), iterator);
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        ScorerSupplier scorerSupplier = scorerSupplier(context);
+        if (scorerSupplier == null) {
+          return null;
+        }
+        return scorerSupplier.get(false);
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/java/org/apache/lucene/search/ScorerSupplier.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/ScorerSupplier.java b/lucene/core/src/java/org/apache/lucene/search/ScorerSupplier.java
new file mode 100644
index 0000000..3f6906a
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/ScorerSupplier.java
@@ -0,0 +1,47 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+
+/**
+ * A supplier of {@link Scorer}. This allows to get an estimate of the cost before
+ * building the {@link Scorer}.
+ */
+public abstract class ScorerSupplier {
+
+  /**
+   * Get the {@link Scorer}. This may not return {@code null} and must be called
+   * at most once.
+   * @param randomAccess A hint about the expected usage of the {@link Scorer}.
+   * If {@link DocIdSetIterator#advance} or {@link TwoPhaseIterator} will be
+   * used to check whether given doc ids match, then pass {@code true}.
+   * Otherwise if the {@link Scorer} will be mostly used to lead the iteration
+   * using {@link DocIdSetIterator#nextDoc()}, then {@code false} should be
+   * passed. Under doubt, pass {@code false} which usually has a better
+   * worst-case.
+   */
+  public abstract Scorer get(boolean randomAccess) throws IOException;
+
+  /**
+   * Get an estimate of the {@link Scorer} that would be returned by {@link #get}.
+   * This may be a costly operation, so it should only be called if necessary.
+   * @see DocIdSetIterator#cost
+   */
+  public abstract long cost();
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/java/org/apache/lucene/search/Weight.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/Weight.java b/lucene/core/src/java/org/apache/lucene/search/Weight.java
index 40d2aaf..76193e1 100644
--- a/lucene/core/src/java/org/apache/lucene/search/Weight.java
+++ b/lucene/core/src/java/org/apache/lucene/search/Weight.java
@@ -116,6 +116,31 @@ public abstract class Weight {
   public abstract Scorer scorer(LeafReaderContext context) throws IOException;
 
   /**
+   * Optional method.
+   * Get a {@link ScorerSupplier}, which allows to know the cost of the {@link Scorer}
+   * before building it. The default implementation calls {@link #scorer} and
+   * builds a {@link ScorerSupplier} wrapper around it.
+   * @see #scorer
+   */
+  public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
+    final Scorer scorer = scorer(context);
+    if (scorer == null) {
+      return null;
+    }
+    return new ScorerSupplier() {
+      @Override
+      public Scorer get(boolean randomAccess) {
+        return scorer;
+      }
+
+      @Override
+      public long cost() {
+        return scorer.iterator().cost();
+      }
+    };
+  }
+
+  /**
    * Optional method, to return a {@link BulkScorer} to
    * score the query and send hits to a {@link Collector}.
    * Only queries that have a different top-level approach

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java
index 4f4228d..7778472 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java
@@ -487,6 +487,10 @@ public final class BKDReader implements Accountable {
     intersect(getIntersectState(visitor), minPackedValue, maxPackedValue);
   }
 
+  public long estimatePointCount(IntersectVisitor visitor) {
+    return estimatePointCount(getIntersectState(visitor), minPackedValue, maxPackedValue);
+  }
+
   /** Fast path: this is called when the query box fully encompasses all cells under this node. */
   private void addAll(IntersectState state) throws IOException {
     //System.out.println("R: addAll nodeID=" + nodeID);
@@ -697,6 +701,59 @@ public final class BKDReader implements Accountable {
     }
   }
 
+  private long estimatePointCount(IntersectState state, byte[] cellMinPacked, byte[] cellMaxPacked) {
+
+    /*
+    System.out.println("\nR: intersect nodeID=" + state.index.getNodeID());
+    for(int dim=0;dim<numDims;dim++) {
+      System.out.println("  dim=" + dim + "\n    cellMin=" + new BytesRef(cellMinPacked, dim*bytesPerDim, bytesPerDim) + "\n    cellMax=" + new BytesRef(cellMaxPacked, dim*bytesPerDim, bytesPerDim));
+    }
+    */
+
+    Relation r = state.visitor.compare(cellMinPacked, cellMaxPacked);
+
+    if (r == Relation.CELL_OUTSIDE_QUERY) {
+      // This cell is fully outside of the query shape: stop recursing
+      return 0L;
+    } else if (state.index.isLeafNode()) {
+      // Assume all points match and there are no dups
+      return maxPointsInLeafNode;
+    } else {
+      
+      // Non-leaf node: recurse on the split left and right nodes
+      int splitDim = state.index.getSplitDim();
+      assert splitDim >= 0: "splitDim=" + splitDim;
+      assert splitDim < numDims;
+
+      byte[] splitPackedValue = state.index.getSplitPackedValue();
+      BytesRef splitDimValue = state.index.getSplitDimValue();
+      assert splitDimValue.length == bytesPerDim;
+      //System.out.println("  splitDimValue=" + splitDimValue + " splitDim=" + splitDim);
+
+      // make sure cellMin <= splitValue <= cellMax:
+      assert StringHelper.compare(bytesPerDim, cellMinPacked, splitDim*bytesPerDim, splitDimValue.bytes, splitDimValue.offset) <= 0: "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numDims=" + numDims;
+      assert StringHelper.compare(bytesPerDim, cellMaxPacked, splitDim*bytesPerDim, splitDimValue.bytes, splitDimValue.offset) >= 0: "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numDims=" + numDims;
+
+      // Recurse on left sub-tree:
+      System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedBytesLength);
+      System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
+      state.index.pushLeft();
+      final long leftCost = estimatePointCount(state, cellMinPacked, splitPackedValue);
+      state.index.pop();
+
+      // Restore the split dim value since it may have been overwritten while recursing:
+      System.arraycopy(splitPackedValue, splitDim*bytesPerDim, splitDimValue.bytes, splitDimValue.offset, bytesPerDim);
+
+      // Recurse on right sub-tree:
+      System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedBytesLength);
+      System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
+      state.index.pushRight();
+      final long rightCost = estimatePointCount(state, splitPackedValue, cellMaxPacked);
+      state.index.pop();
+      return leftCost + rightCost;
+    }
+  }
+
   @Override
   public long ramBytesUsed() {
     if (packedIndex != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/test/org/apache/lucene/search/TestBoolean2ScorerSupplier.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBoolean2ScorerSupplier.java b/lucene/core/src/test/org/apache/lucene/search/TestBoolean2ScorerSupplier.java
new file mode 100644
index 0000000..e8fc63f
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBoolean2ScorerSupplier.java
@@ -0,0 +1,339 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.EnumMap;
+import java.util.Map;
+
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+
+public class TestBoolean2ScorerSupplier extends LuceneTestCase {
+
+  private static class FakeScorer extends Scorer {
+
+    private final DocIdSetIterator it;
+
+    FakeScorer(long cost) {
+      super(null);
+      this.it = DocIdSetIterator.all(Math.toIntExact(cost));
+    }
+
+    @Override
+    public int docID() {
+      return it.docID();
+    }
+
+    @Override
+    public float score() throws IOException {
+      return 1;
+    }
+
+    @Override
+    public int freq() throws IOException {
+      return 1;
+    }
+
+    @Override
+    public DocIdSetIterator iterator() {
+      return it;
+    }
+
+    @Override
+    public String toString() {
+      return "FakeScorer(cost=" + it.cost() + ")";
+    }
+
+  }
+
+  private static class FakeScorerSupplier extends ScorerSupplier {
+
+    private final long cost;
+    private final Boolean randomAccess;
+
+    FakeScorerSupplier(long cost) {
+      this.cost = cost;
+      this.randomAccess = null;
+    }
+
+    FakeScorerSupplier(long cost, boolean randomAccess) {
+      this.cost = cost;
+      this.randomAccess = randomAccess;
+    }
+
+    @Override
+    public Scorer get(boolean randomAccess) throws IOException {
+      if (this.randomAccess != null) {
+        assertEquals(this.toString(), this.randomAccess, randomAccess);
+      }
+      return new FakeScorer(cost);
+    }
+
+    @Override
+    public long cost() {
+      return cost;
+    }
+    
+    @Override
+    public String toString() {
+      return "FakeLazyScorer(cost=" + cost + ",randomAccess=" + randomAccess + ")";
+    }
+
+  }
+
+  private static Boolean2ScorerSupplier scorerSupplier(Map<Occur, Collection<ScorerSupplier>> subs,
+      boolean needsScores, int minShouldMatch) {
+    int maxCoord = subs.get(Occur.SHOULD).size() + subs.get(Occur.MUST).size();
+    float[] coords = new float[maxCoord];
+    Arrays.fill(coords, 1f);
+    return new Boolean2ScorerSupplier(null, subs, true, coords, maxCoord, needsScores, minShouldMatch);
+  }
+
+  public void testConjunctionCost() {
+    Map<Occur, Collection<ScorerSupplier>> subs = new EnumMap<>(Occur.class);
+    for (Occur occur : Occur.values()) {
+      subs.put(occur, new ArrayList<>());
+    }
+
+    subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST))).add(new FakeScorerSupplier(42));
+    assertEquals(42, scorerSupplier(subs, random().nextBoolean(), 0).cost());
+
+    subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST))).add(new FakeScorerSupplier(12));
+    assertEquals(12, scorerSupplier(subs, random().nextBoolean(), 0).cost());
+
+    subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST))).add(new FakeScorerSupplier(20));
+    assertEquals(12, scorerSupplier(subs, random().nextBoolean(), 0).cost());
+  }
+
+  public void testDisjunctionCost() throws IOException {
+    Map<Occur, Collection<ScorerSupplier>> subs = new EnumMap<>(Occur.class);
+    for (Occur occur : Occur.values()) {
+      subs.put(occur, new ArrayList<>());
+    }
+
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42));
+    ScorerSupplier s = scorerSupplier(subs, random().nextBoolean(), 0);
+    assertEquals(42, s.cost());
+    assertEquals(42, s.get(random().nextBoolean()).iterator().cost());
+
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12));
+    s = scorerSupplier(subs, random().nextBoolean(), 0);
+    assertEquals(42 + 12, s.cost());
+    assertEquals(42 + 12, s.get(random().nextBoolean()).iterator().cost());
+
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(20));
+    s = scorerSupplier(subs, random().nextBoolean(), 0);
+    assertEquals(42 + 12 + 20, s.cost());
+    assertEquals(42 + 12 + 20, s.get(random().nextBoolean()).iterator().cost());
+  }
+
+  public void testDisjunctionWithMinShouldMatchCost() throws IOException {
+    Map<Occur, Collection<ScorerSupplier>> subs = new EnumMap<>(Occur.class);
+    for (Occur occur : Occur.values()) {
+      subs.put(occur, new ArrayList<>());
+    }
+
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42));
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12));
+    ScorerSupplier s = scorerSupplier(subs, random().nextBoolean(), 1);
+    assertEquals(42 + 12, s.cost());
+    assertEquals(42 + 12, s.get(random().nextBoolean()).iterator().cost());
+
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(20));
+    s = scorerSupplier(subs, random().nextBoolean(), 1);
+    assertEquals(42 + 12 + 20, s.cost());
+    assertEquals(42 + 12 + 20, s.get(random().nextBoolean()).iterator().cost());
+    s = scorerSupplier(subs, random().nextBoolean(), 2);
+    assertEquals(12 + 20, s.cost());
+    assertEquals(12 + 20, s.get(random().nextBoolean()).iterator().cost());
+
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30));
+    s = scorerSupplier(subs, random().nextBoolean(), 1);
+    assertEquals(42 + 12 + 20 + 30, s.cost());
+    assertEquals(42 + 12 + 20 + 30, s.get(random().nextBoolean()).iterator().cost());
+    s = scorerSupplier(subs, random().nextBoolean(), 2);
+    assertEquals(12 + 20 + 30, s.cost());
+    assertEquals(12 + 20 + 30, s.get(random().nextBoolean()).iterator().cost());
+    s = scorerSupplier(subs, random().nextBoolean(), 3);
+    assertEquals(12 + 20, s.cost());
+    assertEquals(12 + 20, s.get(random().nextBoolean()).iterator().cost());
+  }
+
+  public void testDuelCost() throws Exception {
+    final int iters = atLeast(1000);
+    for (int iter = 0; iter < iters; ++iter) {
+      Map<Occur, Collection<ScorerSupplier>> subs = new EnumMap<>(Occur.class);
+      for (Occur occur : Occur.values()) {
+        subs.put(occur, new ArrayList<>());
+      }
+      int numClauses = TestUtil.nextInt(random(), 1, 10);
+      int numShoulds = 0;
+      int numRequired = 0;
+      for (int j = 0; j < numClauses; ++j) {
+        Occur occur = RandomPicks.randomFrom(random(), Occur.values());
+        subs.get(occur).add(new FakeScorerSupplier(random().nextInt(100)));
+        if (occur == Occur.SHOULD) {
+          ++numShoulds;
+        } else if (occur == Occur.FILTER || occur == Occur.MUST) {
+          numRequired++;
+        }
+      }
+      boolean needsScores = random().nextBoolean();
+      if (needsScores == false && numRequired > 0) {
+        numClauses -= numShoulds;
+        numShoulds = 0;
+        subs.get(Occur.SHOULD).clear();
+      }
+      if (numShoulds + numRequired == 0) {
+        // only negative clauses, invalid
+        continue;
+      }
+      int minShouldMatch = numShoulds == 0 ? 0 : TestUtil.nextInt(random(), 0, numShoulds - 1);
+      Boolean2ScorerSupplier supplier = scorerSupplier(subs, needsScores, minShouldMatch);
+      long cost1 = supplier.cost();
+      long cost2 = supplier.get(false).iterator().cost();
+      assertEquals("clauses=" + subs + ", minShouldMatch=" + minShouldMatch, cost1, cost2);
+    }
+  }
+
+  // test the tester...
+  public void testFakeScorerSupplier() {
+    FakeScorerSupplier randomAccessSupplier = new FakeScorerSupplier(random().nextInt(100), true);
+    expectThrows(AssertionError.class, () -> randomAccessSupplier.get(false));
+    FakeScorerSupplier sequentialSupplier = new FakeScorerSupplier(random().nextInt(100), false);
+    expectThrows(AssertionError.class, () -> sequentialSupplier.get(true));
+  }
+
+  public void testConjunctionRandomAccess() throws IOException {
+    Map<Occur, Collection<ScorerSupplier>> subs = new EnumMap<>(Occur.class);
+    for (Occur occur : Occur.values()) {
+      subs.put(occur, new ArrayList<>());
+    }
+
+    // If sequential access is required, only the least costly clause does not use random-access
+    subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST))).add(new FakeScorerSupplier(42, true));
+    subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST))).add(new FakeScorerSupplier(12, false));
+    scorerSupplier(subs, random().nextBoolean(), 0).get(false); // triggers assertions as a side-effect
+
+    subs = new EnumMap<>(Occur.class);
+    for (Occur occur : Occur.values()) {
+      subs.put(occur, new ArrayList<>());
+    }
+
+    // If random access is required, then we propagate to sub clauses
+    subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST))).add(new FakeScorerSupplier(42, true));
+    subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST))).add(new FakeScorerSupplier(12, true));
+    scorerSupplier(subs, random().nextBoolean(), 0).get(true); // triggers assertions as a side-effect
+  }
+
+  public void testDisjunctionRandomAccess() throws IOException {
+    // disjunctions propagate
+    for (boolean randomAccess : new boolean[] {false, true}) {
+      Map<Occur, Collection<ScorerSupplier>> subs = new EnumMap<>(Occur.class);
+      for (Occur occur : Occur.values()) {
+        subs.put(occur, new ArrayList<>());
+      }
+      subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42, randomAccess));
+      subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, randomAccess));
+      scorerSupplier(subs, random().nextBoolean(), 0).get(randomAccess); // triggers assertions as a side-effect
+    }
+  }
+
+  public void testDisjunctionWithMinShouldMatchRandomAccess() throws IOException {
+    Map<Occur, Collection<ScorerSupplier>> subs = new EnumMap<>(Occur.class);
+    for (Occur occur : Occur.values()) {
+      subs.put(occur, new ArrayList<>());
+    }
+
+    // Only the most costly clause uses random-access in that case:
+    // most of time, we will find agreement between the 2 least costly
+    // clauses and only then check whether the 3rd one matches too
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42, true));
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, false));
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, false));
+    scorerSupplier(subs, random().nextBoolean(), 2).get(false); // triggers assertions as a side-effect
+
+    subs = new EnumMap<>(Occur.class);
+    for (Occur occur : Occur.values()) {
+      subs.put(occur, new ArrayList<>());
+    }
+
+    // When random-access is true, just propagate
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42, true));
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, true));
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, true));
+    scorerSupplier(subs, random().nextBoolean(), 2).get(true); // triggers assertions as a side-effect
+
+    subs = new EnumMap<>(Occur.class);
+    for (Occur occur : Occur.values()) {
+      subs.put(occur, new ArrayList<>());
+    }
+
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42, true));
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, false));
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, false));
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(20, false));
+    scorerSupplier(subs, random().nextBoolean(), 2).get(false); // triggers assertions as a side-effect
+
+    subs = new EnumMap<>(Occur.class);
+    for (Occur occur : Occur.values()) {
+      subs.put(occur, new ArrayList<>());
+    }
+
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42, true));
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, false));
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, true));
+    subs.get(Occur.SHOULD).add(new FakeScorerSupplier(20, false));
+    scorerSupplier(subs, random().nextBoolean(), 3).get(false); // triggers assertions as a side-effect
+  }
+
+  public void testProhibitedRandomAccess() throws IOException {
+    for (boolean randomAccess : new boolean[] {false, true}) {
+      Map<Occur, Collection<ScorerSupplier>> subs = new EnumMap<>(Occur.class);
+      for (Occur occur : Occur.values()) {
+        subs.put(occur, new ArrayList<>());
+      }
+
+      // The MUST_NOT clause always uses random-access
+      subs.get(Occur.MUST).add(new FakeScorerSupplier(42, randomAccess));
+      subs.get(Occur.MUST_NOT).add(new FakeScorerSupplier(TestUtil.nextInt(random(), 1, 100), true));
+      scorerSupplier(subs, random().nextBoolean(), 0).get(randomAccess); // triggers assertions as a side-effect
+    }
+  }
+
+  public void testMixedRandomAccess() throws IOException {
+    for (boolean randomAccess : new boolean[] {false, true}) {
+      Map<Occur, Collection<ScorerSupplier>> subs = new EnumMap<>(Occur.class);
+      for (Occur occur : Occur.values()) {
+        subs.put(occur, new ArrayList<>());
+      }
+
+      // The SHOULD clause always uses random-access if there is a MUST clause
+      subs.get(Occur.MUST).add(new FakeScorerSupplier(42, randomAccess));
+      subs.get(Occur.SHOULD).add(new FakeScorerSupplier(TestUtil.nextInt(random(), 1, 100), true));
+      scorerSupplier(subs, true, 0).get(randomAccess); // triggers assertions as a side-effect
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
index 443246b..3d89775 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
@@ -206,8 +206,8 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
           "    MUST ConstantScoreScorer\n" +
           "    MUST MinShouldMatchSumScorer\n" +
           "            SHOULD TermScorer body:nutch\n" +
-          "            SHOULD TermScorer body:web\n" +
-          "            SHOULD TermScorer body:crawler",
+          "            SHOULD TermScorer body:crawler\n" +
+          "            SHOULD TermScorer body:web",
           summary);
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/test/org/apache/lucene/search/TestConjunctionDISI.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestConjunctionDISI.java b/lucene/core/src/test/org/apache/lucene/search/TestConjunctionDISI.java
index 9835f35..8df4db6 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestConjunctionDISI.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestConjunctionDISI.java
@@ -341,7 +341,7 @@ public class TestConjunctionDISI extends LuceneTestCase {
         List<Scorer> subIterators = scorers.subList(subSeqStart, subSeqEnd);
         Scorer subConjunction;
         if (wrapWithScorer) {
-          subConjunction = new ConjunctionScorer(null, subIterators, Collections.emptyList());
+          subConjunction = new ConjunctionScorer(null, subIterators, Collections.emptyList(), 1f);
         } else {
           subConjunction = new ConstantScoreScorer(null, 0f, ConjunctionDISI.intersectScorers(subIterators));
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/test/org/apache/lucene/search/TestFilterWeight.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestFilterWeight.java b/lucene/core/src/test/org/apache/lucene/search/TestFilterWeight.java
index cfa01bf..b58fe1b 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestFilterWeight.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestFilterWeight.java
@@ -18,6 +18,7 @@ package org.apache.lucene.search;
 
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
+import java.util.Arrays;
 
 import org.apache.lucene.util.LuceneTestCase;
 import org.junit.Test;
@@ -35,7 +36,7 @@ public class TestFilterWeight extends LuceneTestCase {
       final int modifiers = superClassMethod.getModifiers();
       if (Modifier.isFinal(modifiers)) continue;
       if (Modifier.isStatic(modifiers)) continue;
-      if (superClassMethod.getName().equals("bulkScorer")) {
+      if (Arrays.asList("bulkScorer", "scorerSupplier").contains(superClassMethod.getName())) {
         try {
           final Method subClassMethod = subClass.getDeclaredMethod(
               superClassMethod.getName(),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java b/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java
index 62e85fe..803a7a7 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java
@@ -312,6 +312,11 @@ public class TestDocIdSetBuilder extends LuceneTestCase {
     }
 
     @Override
+    public long estimatePointCount(String fieldName, IntersectVisitor visitor) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
     public byte[] getMinPackedValue(String fieldName) throws IOException {
       throw new UnsupportedOperationException();
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/test/org/apache/lucene/util/bkd/Test2BBKDPoints.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/bkd/Test2BBKDPoints.java b/lucene/core/src/test/org/apache/lucene/util/bkd/Test2BBKDPoints.java
index a89a184..9162d5e 100644
--- a/lucene/core/src/test/org/apache/lucene/util/bkd/Test2BBKDPoints.java
+++ b/lucene/core/src/test/org/apache/lucene/util/bkd/Test2BBKDPoints.java
@@ -133,6 +133,12 @@ public class Test2BBKDPoints extends LuceneTestCase {
     }
 
     @Override
+    public long estimatePointCount(String fieldNameIn, IntersectVisitor visitor) {
+      verifyFieldName(fieldNameIn);
+      return bkdReader.estimatePointCount(visitor);
+    }
+
+    @Override
     public byte[] getMinPackedValue(String fieldNameIn) throws IOException {
       verifyFieldName(fieldNameIn);
       return bkdReader.getMinPackedValue();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/core/src/test/org/apache/lucene/util/bkd/TestMutablePointsReaderUtils.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/bkd/TestMutablePointsReaderUtils.java b/lucene/core/src/test/org/apache/lucene/util/bkd/TestMutablePointsReaderUtils.java
index 4616ce3..cd94719 100644
--- a/lucene/core/src/test/org/apache/lucene/util/bkd/TestMutablePointsReaderUtils.java
+++ b/lucene/core/src/test/org/apache/lucene/util/bkd/TestMutablePointsReaderUtils.java
@@ -236,6 +236,11 @@ public class TestMutablePointsReaderUtils extends LuceneTestCase {
     }
 
     @Override
+    public long estimatePointCount(String fieldName, IntersectVisitor visitor) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
     public byte[] getMinPackedValue(String fieldName) throws IOException {
       throw new UnsupportedOperationException();
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
----------------------------------------------------------------------
diff --git a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
index 9037dfa..362e7fb 100644
--- a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
+++ b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
@@ -1510,6 +1510,11 @@ public class MemoryIndex {
       }
 
       @Override
+      public long estimatePointCount(String fieldName, IntersectVisitor visitor) {
+        return 1;
+      }
+
+      @Override
       public byte[] getMinPackedValue(String fieldName) throws IOException {
         Info info = fields.get(fieldName);
         if (info == null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesRangeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesRangeQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesRangeQuery.java
index d603040..b71ebb2 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesRangeQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesRangeQuery.java
@@ -23,8 +23,10 @@ import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PointValues;
 import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.Terms;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 
@@ -34,10 +36,11 @@ import org.apache.lucene.util.BytesRef;
  * dense case where most documents match this query, it <b>might</b> be as
  * fast or faster than a regular {@link PointRangeQuery}.
  *
- * <p>
- * <b>NOTE</b>: be very careful using this query: it is
- * typically much slower than using {@code TermsQuery},
- * but in certain specialized cases may be faster.
+ * <b>NOTE:</b> This query is typically best used within a
+ * {@link IndexOrDocValuesQuery} alongside a query that uses an indexed
+ * structure such as {@link PointValues points} or {@link Terms terms},
+ * which allows to run the query on doc values when that would be more
+ * efficient, and using an index otherwise.
  *
  * @lucene.experimental
  */