You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2023/01/16 14:38:15 UTC

[lucene] branch main updated: Graduate DocValuesNumbersQuery from lucene/sandbox to newSlowSetQuery() (#12087)

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

rmuir pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/lucene.git


The following commit(s) were added to refs/heads/main by this push:
     new 4fe8424925c Graduate DocValuesNumbersQuery from lucene/sandbox to newSlowSetQuery() (#12087)
4fe8424925c is described below

commit 4fe8424925ca404d335fa41d261545d3182c22fa
Author: Robert Muir <rm...@apache.org>
AuthorDate: Mon Jan 16 09:38:08 2023 -0500

    Graduate DocValuesNumbersQuery from lucene/sandbox to newSlowSetQuery() (#12087)
    
    Clean up this query a bit and support:
    * NumericDocValuesField.newSlowSetQuery()
    * SortedNumericDocValuesField.newSlowSetQuery()
    
    This complements the existing docvalues-based range queries, with a set query.
    
    Add ScorerSupplier/cost estimation support to PointInSetQuery
    Add newSetQuery() to IntField/LongField/DoubleField/FloatField, that uses IndexOrDocValuesQuery
---
 lucene/CHANGES.txt                                 |   4 +
 .../org/apache/lucene/document/DoubleField.java    |  22 +++
 .../org/apache/lucene/document/FloatField.java     |  22 +++
 .../java/org/apache/lucene/document/IntField.java  |  22 +++
 .../java/org/apache/lucene/document/LongField.java |  19 ++
 .../org/apache/lucene/document}/LongHashSet.java   |   4 +-
 .../lucene/document/NumericDocValuesField.java     |  36 ++++
 .../document/SortedNumericDocValuesField.java      |  38 ++++
 .../document/SortedNumericDocValuesSetQuery.java}  | 132 +++++++-------
 .../org/apache/lucene/search/PointInSetQuery.java  |  84 +++++++--
 .../apache/lucene/document}/TestLongHashSet.java   |  12 +-
 .../apache/lucene/search/TestDocValuesQueries.java |  97 ++++++++++
 .../sandbox/search/TestDocValuesNumbersQuery.java  | 203 ---------------------
 13 files changed, 405 insertions(+), 290 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index d1252ba96c1..75cacb48c81 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -153,6 +153,10 @@ API Changes
 * GITHUB#12038: Deprecate non-NRT replication support.
   Please migrate to org.apache.lucene.replicator.nrt instead. (Robert Muir)
 
+* GITHUB#12087: Move DocValuesNumbersQuery from sandbox to NumericDocValuesField#newSlowSetQuery
+  and SortedNumericDocValuesField#newSlowSetQuery. IntField, LongField, FloatField, and DoubleField
+  implement newSetQuery with best-practice use of IndexOrDocValuesQuery. (Robert Muir)
+
 New Features
 ---------------------
 * GITHUB#11795: Add ByteWritesTrackingDirectoryWrapper to expose metrics for bytes merged, flushed, and overall
diff --git a/lucene/core/src/java/org/apache/lucene/document/DoubleField.java b/lucene/core/src/java/org/apache/lucene/document/DoubleField.java
index eb9c4f5e6ef..dc4b4ce6e72 100644
--- a/lucene/core/src/java/org/apache/lucene/document/DoubleField.java
+++ b/lucene/core/src/java/org/apache/lucene/document/DoubleField.java
@@ -38,6 +38,7 @@ import org.apache.lucene.util.NumericUtils;
  * <ul>
  *   <li>{@link #newExactQuery(String, double)} for matching an exact 1D point.
  *   <li>{@link #newRangeQuery(String, double, double)} for matching a 1D range.
+ *   <li>{@link #newSetQuery(String, double...)} for matching a 1D set.
  * </ul>
  *
  * @see PointValues
@@ -127,6 +128,27 @@ public final class DoubleField extends Field {
             NumericUtils.doubleToSortableLong(upperValue)));
   }
 
+  /**
+   * Create a query matching values in a supplied set
+   *
+   * @param field field name. must not be {@code null}.
+   * @param values double values
+   * @throws IllegalArgumentException if {@code field} is null.
+   * @return a query matching documents within this set.
+   */
+  public static Query newSetQuery(String field, double... values) {
+    if (field == null) {
+      throw new IllegalArgumentException("field cannot be null");
+    }
+    long points[] = new long[values.length];
+    for (int i = 0; i < values.length; i++) {
+      points[i] = NumericUtils.doubleToSortableLong(values[i]);
+    }
+    return new IndexOrDocValuesQuery(
+        DoublePoint.newSetQuery(field, values.clone()),
+        SortedNumericDocValuesField.newSlowSetQuery(field, points));
+  }
+
   /**
    * Create a new {@link SortField} for double values.
    *
diff --git a/lucene/core/src/java/org/apache/lucene/document/FloatField.java b/lucene/core/src/java/org/apache/lucene/document/FloatField.java
index d811b7c20a1..437cf57d1ce 100644
--- a/lucene/core/src/java/org/apache/lucene/document/FloatField.java
+++ b/lucene/core/src/java/org/apache/lucene/document/FloatField.java
@@ -38,6 +38,7 @@ import org.apache.lucene.util.NumericUtils;
  * <ul>
  *   <li>{@link #newExactQuery(String, float)} for matching an exact 1D point.
  *   <li>{@link #newRangeQuery(String, float, float)} for matching a 1D range.
+ *   <li>{@link #newSetQuery(String, float...)} for matching a 1D set.
  * </ul>
  *
  * @see PointValues
@@ -127,6 +128,27 @@ public final class FloatField extends Field {
             NumericUtils.floatToSortableInt(upperValue)));
   }
 
+  /**
+   * Create a query matching values in a supplied set
+   *
+   * @param field field name. must not be {@code null}.
+   * @param values float values
+   * @throws IllegalArgumentException if {@code field} is null.
+   * @return a query matching documents within this set.
+   */
+  public static Query newSetQuery(String field, float... values) {
+    if (field == null) {
+      throw new IllegalArgumentException("field cannot be null");
+    }
+    long points[] = new long[values.length];
+    for (int i = 0; i < values.length; i++) {
+      points[i] = NumericUtils.floatToSortableInt(values[i]);
+    }
+    return new IndexOrDocValuesQuery(
+        FloatPoint.newSetQuery(field, values.clone()),
+        SortedNumericDocValuesField.newSlowSetQuery(field, points));
+  }
+
   /**
    * Create a new {@link SortField} for float values.
    *
diff --git a/lucene/core/src/java/org/apache/lucene/document/IntField.java b/lucene/core/src/java/org/apache/lucene/document/IntField.java
index 04c788d7d09..30a9fb64be4 100644
--- a/lucene/core/src/java/org/apache/lucene/document/IntField.java
+++ b/lucene/core/src/java/org/apache/lucene/document/IntField.java
@@ -39,6 +39,7 @@ import org.apache.lucene.util.NumericUtils;
  * <ul>
  *   <li>{@link #newExactQuery(String, int)} for matching an exact 1D point.
  *   <li>{@link #newRangeQuery(String, int, int)} for matching a 1D range.
+ *   <li>{@link #newSetQuery(String, int...)} for matching a 1D set.
  * </ul>
  *
  * @see PointValues
@@ -114,6 +115,27 @@ public final class IntField extends Field {
         field, lowerValue, upperValue, fallbackQuery);
   }
 
+  /**
+   * Create a query matching values in a supplied set
+   *
+   * @param field field name. must not be {@code null}.
+   * @param values integer values
+   * @throws IllegalArgumentException if {@code field} is null.
+   * @return a query matching documents within this set.
+   */
+  public static Query newSetQuery(String field, int... values) {
+    if (field == null) {
+      throw new IllegalArgumentException("field cannot be null");
+    }
+    long points[] = new long[values.length];
+    for (int i = 0; i < values.length; i++) {
+      points[i] = values[i];
+    }
+    return new IndexOrDocValuesQuery(
+        IntPoint.newSetQuery(field, values.clone()),
+        SortedNumericDocValuesField.newSlowSetQuery(field, points));
+  }
+
   /**
    * Create a new {@link SortField} for int values.
    *
diff --git a/lucene/core/src/java/org/apache/lucene/document/LongField.java b/lucene/core/src/java/org/apache/lucene/document/LongField.java
index 3d4994147a0..af4dfd4bc81 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LongField.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LongField.java
@@ -42,6 +42,7 @@ import org.apache.lucene.util.NumericUtils;
  * <ul>
  *   <li>{@link #newExactQuery(String, long)} for matching an exact 1D point.
  *   <li>{@link #newRangeQuery(String, long, long)} for matching a 1D range.
+ *   <li>{@link #newSetQuery(String, long...)} for matching a 1D set.
  * </ul>
  *
  * @see PointValues
@@ -117,6 +118,24 @@ public final class LongField extends Field {
         field, lowerValue, upperValue, fallbackQuery);
   }
 
+  /**
+   * Create a query matching values in a supplied set
+   *
+   * @param field field name. must not be {@code null}.
+   * @param values long values
+   * @throws IllegalArgumentException if {@code field} is null.
+   * @return a query matching documents within this set.
+   */
+  public static Query newSetQuery(String field, long... values) {
+    if (field == null) {
+      throw new IllegalArgumentException("field cannot be null");
+    }
+    long points[] = values.clone();
+    return new IndexOrDocValuesQuery(
+        LongPoint.newSetQuery(field, points),
+        SortedNumericDocValuesField.newSlowSetQuery(field, points));
+  }
+
   /**
    * Create a new {@link SortField} for long values.
    *
diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/LongHashSet.java b/lucene/core/src/java/org/apache/lucene/document/LongHashSet.java
similarity index 98%
rename from lucene/sandbox/src/java/org/apache/lucene/sandbox/search/LongHashSet.java
rename to lucene/core/src/java/org/apache/lucene/document/LongHashSet.java
index 4133e03da4f..402009a3acc 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/LongHashSet.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LongHashSet.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.lucene.sandbox.search;
+package org.apache.lucene.document;
 
 import java.util.AbstractSet;
 import java.util.Arrays;
@@ -36,7 +36,7 @@ final class LongHashSet extends AbstractSet<Long> implements Accountable {
   final int size;
   final int hashCode;
 
-  LongHashSet(long... values) {
+  LongHashSet(long[] values) {
     int tableSize = Math.toIntExact(values.length * 3L / 2);
     tableSize = 1 << PackedInts.bitsRequired(tableSize); // make it a power of 2
     assert tableSize >= values.length * 3L / 2;
diff --git a/lucene/core/src/java/org/apache/lucene/document/NumericDocValuesField.java b/lucene/core/src/java/org/apache/lucene/document/NumericDocValuesField.java
index 3fc36a416bb..dc14fea8471 100644
--- a/lucene/core/src/java/org/apache/lucene/document/NumericDocValuesField.java
+++ b/lucene/core/src/java/org/apache/lucene/document/NumericDocValuesField.java
@@ -83,6 +83,11 @@ public class NumericDocValuesField extends Field {
    * slow if they are not ANDed with a selective query. As a consequence, they are best used wrapped
    * in an {@link IndexOrDocValuesQuery}, alongside a range query that executes on points, such as
    * {@link LongPoint#newRangeQuery}.
+   *
+   * @see IntField#newRangeQuery
+   * @see LongField#newRangeQuery
+   * @see FloatField#newRangeQuery
+   * @see DoubleField#newRangeQuery
    */
   public static Query newSlowRangeQuery(String field, long lowerValue, long upperValue) {
     return new SortedNumericDocValuesRangeQuery(field, lowerValue, upperValue) {
@@ -97,6 +102,32 @@ public class NumericDocValuesField extends Field {
     };
   }
 
+  /**
+   * Create a query matching any of the specified values.
+   *
+   * <p><b>NOTE</b>: Such queries cannot efficiently advance to the next match, which makes them
+   * slow if they are not ANDed with a selective query. As a consequence, they are best used wrapped
+   * in an {@link IndexOrDocValuesQuery}, alongside a set query that executes on points, such as
+   * {@link LongPoint#newSetQuery}.
+   *
+   * @see IntField#newSetQuery
+   * @see LongField#newSetQuery
+   * @see FloatField#newSetQuery
+   * @see DoubleField#newSetQuery
+   */
+  public static Query newSlowSetQuery(String field, long... values) {
+    return new SortedNumericDocValuesSetQuery(field, values.clone()) {
+      @Override
+      SortedNumericDocValues getValues(LeafReader reader, String field) throws IOException {
+        NumericDocValues values = reader.getNumericDocValues(field);
+        if (values == null) {
+          return null;
+        }
+        return DocValues.singleton(values);
+      }
+    };
+  }
+
   /**
    * Create a query for matching an exact long value.
    *
@@ -104,6 +135,11 @@ public class NumericDocValuesField extends Field {
    * slow if they are not ANDed with a selective query. As a consequence, they are best used wrapped
    * in an {@link IndexOrDocValuesQuery}, alongside a range query that executes on points, such as
    * {@link LongPoint#newExactQuery}.
+   *
+   * @see IntField#newExactQuery
+   * @see LongField#newExactQuery
+   * @see FloatField#newExactQuery
+   * @see DoubleField#newExactQuery
    */
   public static Query newSlowExactQuery(String field, long value) {
     return newSlowRangeQuery(field, value, value);
diff --git a/lucene/core/src/java/org/apache/lucene/document/SortedNumericDocValuesField.java b/lucene/core/src/java/org/apache/lucene/document/SortedNumericDocValuesField.java
index 2cfda1ff94f..0548c3f516f 100644
--- a/lucene/core/src/java/org/apache/lucene/document/SortedNumericDocValuesField.java
+++ b/lucene/core/src/java/org/apache/lucene/document/SortedNumericDocValuesField.java
@@ -81,6 +81,11 @@ public class SortedNumericDocValuesField extends Field {
    * slow if they are not ANDed with a selective query. As a consequence, they are best used wrapped
    * in an {@link IndexOrDocValuesQuery}, alongside a range query that executes on points, such as
    * {@link LongPoint#newRangeQuery}.
+   *
+   * @see IntField#newRangeQuery
+   * @see LongField#newRangeQuery
+   * @see FloatField#newRangeQuery
+   * @see DoubleField#newRangeQuery
    */
   public static Query newSlowRangeQuery(String field, long lowerValue, long upperValue) {
     return new SortedNumericDocValuesRangeQuery(field, lowerValue, upperValue) {
@@ -97,6 +102,34 @@ public class SortedNumericDocValuesField extends Field {
     };
   }
 
+  /**
+   * Create a query matching any of the specified values.
+   *
+   * <p><b>NOTE</b>: Such queries cannot efficiently advance to the next match, which makes them
+   * slow if they are not ANDed with a selective query. As a consequence, they are best used wrapped
+   * in an {@link IndexOrDocValuesQuery}, alongside a set query that executes on points, such as
+   * {@link LongPoint#newSetQuery}.
+   *
+   * @see IntField#newSetQuery
+   * @see LongField#newSetQuery
+   * @see FloatField#newSetQuery
+   * @see DoubleField#newSetQuery
+   */
+  public static Query newSlowSetQuery(String field, long... values) {
+    return new SortedNumericDocValuesSetQuery(field, values.clone()) {
+      @Override
+      SortedNumericDocValues getValues(LeafReader reader, String field) throws IOException {
+        FieldInfo info = reader.getFieldInfos().fieldInfo(field);
+        if (info == null) {
+          // Queries have some optimizations when one sub scorer returns null rather
+          // than a scorer that does not match any documents
+          return null;
+        }
+        return DocValues.getSortedNumeric(reader, field);
+      }
+    };
+  }
+
   /**
    * Create a query for matching an exact long value.
    *
@@ -106,6 +139,11 @@ public class SortedNumericDocValuesField extends Field {
    * slow if they are not ANDed with a selective query. As a consequence, they are best used wrapped
    * in an {@link IndexOrDocValuesQuery}, alongside a range query that executes on points, such as
    * {@link LongPoint#newExactQuery}.
+   *
+   * @see IntField#newExactQuery
+   * @see LongField#newExactQuery
+   * @see FloatField#newExactQuery
+   * @see DoubleField#newExactQuery
    */
   public static Query newSlowExactQuery(String field, long value) {
     return newSlowRangeQuery(field, value, value);
diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/DocValuesNumbersQuery.java b/lucene/core/src/java/org/apache/lucene/document/SortedNumericDocValuesSetQuery.java
similarity index 53%
rename from lucene/sandbox/src/java/org/apache/lucene/sandbox/search/DocValuesNumbersQuery.java
rename to lucene/core/src/java/org/apache/lucene/document/SortedNumericDocValuesSetQuery.java
index 726c08b1aae..574b9eaab82 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/DocValuesNumbersQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/document/SortedNumericDocValuesSetQuery.java
@@ -14,22 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.lucene.sandbox.search;
+package org.apache.lucene.document;
 
 import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
 import java.util.Objects;
-import java.util.Set;
-import org.apache.lucene.document.NumericDocValuesField;
-import org.apache.lucene.document.SortedNumericDocValuesField;
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.search.ConstantScoreScorer;
 import org.apache.lucene.search.ConstantScoreWeight;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryVisitor;
 import org.apache.lucene.search.ScoreMode;
@@ -39,44 +36,26 @@ import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
-/**
- * Like {@link DocValuesTermsQuery}, but this query only runs on a long {@link
- * NumericDocValuesField} or a {@link SortedNumericDocValuesField}, matching all documents whose
- * value in the specified field is contained in the provided set of long values.
- *
- * <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.
- *
- * @lucene.experimental
- */
-public class DocValuesNumbersQuery extends Query implements Accountable {
+/** Similar to SortedNumericDocValuesRangeQuery but for a set */
+abstract class SortedNumericDocValuesSetQuery extends Query implements Accountable {
   private static final long BASE_RAM_BYTES =
-      RamUsageEstimator.shallowSizeOfInstance(DocValuesNumbersQuery.class);
+      RamUsageEstimator.shallowSizeOfInstance(SortedNumericDocValuesSetQuery.class);
 
   private final String field;
   private final LongHashSet numbers;
 
-  public DocValuesNumbersQuery(String field, long[] numbers) {
+  SortedNumericDocValuesSetQuery(String field, long[] numbers) {
     this.field = Objects.requireNonNull(field);
     this.numbers = new LongHashSet(numbers);
   }
 
-  public DocValuesNumbersQuery(String field, Collection<Long> numbers) {
-    this.field = Objects.requireNonNull(field);
-    this.numbers = new LongHashSet(numbers.stream().mapToLong(Long::longValue).toArray());
-  }
-
-  public DocValuesNumbersQuery(String field, Long... numbers) {
-    this(field, new HashSet<Long>(Arrays.asList(numbers)));
-  }
-
   @Override
   public boolean equals(Object other) {
-    return sameClassAs(other) && equalsTo(getClass().cast(other));
-  }
-
-  private boolean equalsTo(DocValuesNumbersQuery other) {
-    return field.equals(other.field) && numbers.equals(other.numbers);
+    if (sameClassAs(other) == false) {
+      return false;
+    }
+    SortedNumericDocValuesSetQuery that = (SortedNumericDocValuesSetQuery) other;
+    return field.equals(that.field) && numbers.equals(that.numbers);
   }
 
   @Override
@@ -91,14 +70,6 @@ public class DocValuesNumbersQuery extends Query implements Accountable {
     }
   }
 
-  public String getField() {
-    return field;
-  }
-
-  public Set<Long> getNumbers() {
-    return numbers;
-  }
-
   @Override
   public String toString(String defaultField) {
     return new StringBuilder().append(field).append(": ").append(numbers.toString()).toString();
@@ -111,41 +82,68 @@ public class DocValuesNumbersQuery extends Query implements Accountable {
         + RamUsageEstimator.sizeOfObject(numbers);
   }
 
+  @Override
+  public Query rewrite(IndexSearcher indexSearcher) throws IOException {
+    if (numbers.size() == 0) {
+      return new MatchNoDocsQuery();
+    }
+    return super.rewrite(indexSearcher);
+  }
+
+  abstract SortedNumericDocValues getValues(LeafReader reader, String field) throws IOException;
+
   @Override
   public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost)
       throws IOException {
     return new ConstantScoreWeight(this, boost) {
 
+      @Override
+      public boolean isCacheable(LeafReaderContext ctx) {
+        return DocValues.isCacheable(ctx, field);
+      }
+
       @Override
       public Scorer scorer(LeafReaderContext context) throws IOException {
-        final SortedNumericDocValues values = DocValues.getSortedNumeric(context.reader(), field);
-        return new ConstantScoreScorer(
-            this,
-            score(),
-            scoreMode,
-            new TwoPhaseIterator(values) {
-
-              @Override
-              public boolean matches() throws IOException {
-                int count = values.docValueCount();
-                for (int i = 0; i < count; i++) {
-                  if (numbers.contains(values.nextValue())) {
-                    return true;
+        SortedNumericDocValues values = getValues(context.reader(), field);
+        if (values == null) {
+          return null;
+        }
+        final NumericDocValues singleton = DocValues.unwrapSingleton(values);
+        final TwoPhaseIterator iterator;
+        if (singleton != null) {
+          iterator =
+              new TwoPhaseIterator(singleton) {
+                @Override
+                public boolean matches() throws IOException {
+                  return numbers.contains(singleton.longValue());
+                }
+
+                @Override
+                public float matchCost() {
+                  return 5; // lookup in the set
+                }
+              };
+        } else {
+          iterator =
+              new TwoPhaseIterator(values) {
+                @Override
+                public boolean matches() throws IOException {
+                  int count = values.docValueCount();
+                  for (int i = 0; i < count; i++) {
+                    if (numbers.contains(values.nextValue())) {
+                      return true;
+                    }
                   }
+                  return false;
                 }
-                return false;
-              }
-
-              @Override
-              public float matchCost() {
-                return 5; // lookup in the set
-              }
-            });
-      }
 
-      @Override
-      public boolean isCacheable(LeafReaderContext ctx) {
-        return true;
+                @Override
+                public float matchCost() {
+                  return 5; // lookup in the set
+                }
+              };
+        }
+        return new ConstantScoreScorer(this, score(), scoreMode, iterator);
       }
     };
   }
diff --git a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
index 13533cf25c7..d2e0f44495c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
@@ -17,6 +17,7 @@
 package org.apache.lucene.search;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.AbstractCollection;
 import java.util.Arrays;
 import java.util.Collection;
@@ -146,6 +147,16 @@ public abstract class PointInSetQuery extends Query implements Accountable {
 
       @Override
       public Scorer scorer(LeafReaderContext context) throws IOException {
+        ScorerSupplier scorerSupplier = scorerSupplier(context);
+        if (scorerSupplier == null) {
+          return null;
+        }
+        return scorerSupplier.get(Long.MAX_VALUE);
+      }
+
+      @Override
+      public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
+        final Weight weight = this;
         LeafReader reader = context.reader();
 
         PointValues values = reader.getPointValues(field);
@@ -173,29 +184,78 @@ public abstract class PointInSetQuery extends Query implements Accountable {
                   + bytesPerDim);
         }
 
-        DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
-
         if (numDims == 1) {
-
           // We optimize this common case, effectively doing a merge sort of the indexed values vs
           // the queried set:
-          values.intersect(new MergePointVisitor(sortedPackedPoints, result));
+          return new ScorerSupplier() {
+            long cost = -1; // calculate lazily, only once
+
+            @Override
+            public Scorer get(long leadCost) throws IOException {
+              DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
+              values.intersect(new MergePointVisitor(sortedPackedPoints, result));
+              DocIdSetIterator iterator = result.build().iterator();
+              return new ConstantScoreScorer(weight, score(), scoreMode, iterator);
+            }
 
+            @Override
+            public long cost() {
+              try {
+                if (cost == -1) {
+                  // Computing the cost may be expensive, so only do it if necessary
+                  DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
+                  cost = values.estimateDocCount(new MergePointVisitor(sortedPackedPoints, result));
+                  assert cost >= 0;
+                }
+                return cost;
+              } catch (IOException e) {
+                throw new UncheckedIOException(e);
+              }
+            }
+          };
         } else {
           // NOTE: this is naive implementation, where for each point we re-walk the KD tree to
           // intersect.  We could instead do a similar
           // optimization as the 1D case, but I think it'd mean building a query-time KD tree so we
           // could efficiently intersect against the
           // index, which is probably tricky!
-          SinglePointVisitor visitor = new SinglePointVisitor(result);
-          TermIterator iterator = sortedPackedPoints.iterator();
-          for (BytesRef point = iterator.next(); point != null; point = iterator.next()) {
-            visitor.setPoint(point);
-            values.intersect(visitor);
-          }
-        }
 
-        return new ConstantScoreScorer(this, score(), scoreMode, result.build().iterator());
+          return new ScorerSupplier() {
+            long cost = -1; // calculate lazily, only once
+
+            @Override
+            public Scorer get(long leadCost) throws IOException {
+              DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
+              SinglePointVisitor visitor = new SinglePointVisitor(result);
+              TermIterator iterator = sortedPackedPoints.iterator();
+              for (BytesRef point = iterator.next(); point != null; point = iterator.next()) {
+                visitor.setPoint(point);
+                values.intersect(visitor);
+              }
+              return new ConstantScoreScorer(weight, score(), scoreMode, result.build().iterator());
+            }
+
+            @Override
+            public long cost() {
+              try {
+                if (cost == -1) {
+                  DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
+                  SinglePointVisitor visitor = new SinglePointVisitor(result);
+                  TermIterator iterator = sortedPackedPoints.iterator();
+                  cost = 0;
+                  for (BytesRef point = iterator.next(); point != null; point = iterator.next()) {
+                    visitor.setPoint(point);
+                    cost += values.estimateDocCount(visitor);
+                  }
+                  assert cost >= 0;
+                }
+                return cost;
+              } catch (IOException e) {
+                throw new UncheckedIOException(e);
+              }
+            }
+          };
+        }
       }
 
       @Override
diff --git a/lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestLongHashSet.java b/lucene/core/src/test/org/apache/lucene/document/TestLongHashSet.java
similarity index 90%
rename from lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestLongHashSet.java
rename to lucene/core/src/test/org/apache/lucene/document/TestLongHashSet.java
index 564f186d9c7..12977373c62 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestLongHashSet.java
+++ b/lucene/core/src/test/org/apache/lucene/document/TestLongHashSet.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.lucene.sandbox.search;
+package org.apache.lucene.document;
 
 import java.util.Arrays;
 import java.util.HashSet;
@@ -53,27 +53,27 @@ public class TestLongHashSet extends LuceneTestCase {
 
   public void testEmpty() {
     Set<Long> set1 = new HashSet<>();
-    LongHashSet set2 = new LongHashSet();
+    LongHashSet set2 = new LongHashSet(new long[] {});
     assertEquals(set1, set2);
   }
 
   public void testOneValue() {
     Set<Long> set1 = new HashSet<>(Arrays.asList(42L));
-    LongHashSet set2 = new LongHashSet(42);
+    LongHashSet set2 = new LongHashSet(new long[] {42L});
     assertEquals(set1, set2);
 
     set1 = new HashSet<>(Arrays.asList(Long.MIN_VALUE));
-    set2 = new LongHashSet(Long.MIN_VALUE);
+    set2 = new LongHashSet(new long[] {Long.MIN_VALUE});
     assertEquals(set1, set2);
   }
 
   public void testTwoValues() {
     Set<Long> set1 = new HashSet<>(Arrays.asList(42L, Long.MAX_VALUE));
-    LongHashSet set2 = new LongHashSet(42, Long.MAX_VALUE);
+    LongHashSet set2 = new LongHashSet(new long[] {42L, Long.MAX_VALUE});
     assertEquals(set1, set2);
 
     set1 = new HashSet<>(Arrays.asList(Long.MIN_VALUE, 42L));
-    set2 = new LongHashSet(Long.MIN_VALUE, 42L);
+    set2 = new LongHashSet(new long[] {Long.MIN_VALUE, 42L});
     assertEquals(set1, set2);
   }
 
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestDocValuesQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestDocValuesQueries.java
index 2f4b74b7f46..a54be70360e 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestDocValuesQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestDocValuesQueries.java
@@ -17,19 +17,27 @@
 package org.apache.lucene.search;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
 import org.apache.lucene.document.LongPoint;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.SortedNumericDocValuesField;
 import org.apache.lucene.document.SortedSetDocValuesField;
+import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.tests.index.RandomIndexWriter;
 import org.apache.lucene.tests.search.QueryUtils;
 import org.apache.lucene.tests.util.LuceneTestCase;
 import org.apache.lucene.tests.util.TestUtil;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.NumericUtils;
 
 public class TestDocValuesQueries extends LuceneTestCase {
@@ -318,4 +326,93 @@ public class TestDocValuesQueries extends LuceneTestCase {
     reader.close();
     dir.close();
   }
+
+  public void testSetEquals() {
+    assertEquals(
+        NumericDocValuesField.newSlowSetQuery("field", 17L, 42L),
+        NumericDocValuesField.newSlowSetQuery("field", 17L, 42L));
+    assertEquals(
+        NumericDocValuesField.newSlowSetQuery("field", 17L, 42L, 32416190071L),
+        NumericDocValuesField.newSlowSetQuery("field", 17L, 32416190071L, 42L));
+    assertFalse(
+        NumericDocValuesField.newSlowSetQuery("field", 42L)
+            .equals(NumericDocValuesField.newSlowSetQuery("field2", 42L)));
+    assertFalse(
+        NumericDocValuesField.newSlowSetQuery("field", 17L, 42L)
+            .equals(NumericDocValuesField.newSlowSetQuery("field", 17L, 32416190071L)));
+  }
+
+  public void testDuelSetVsTermsQuery() throws IOException {
+    final int iters = atLeast(2);
+    for (int iter = 0; iter < iters; ++iter) {
+      final List<Long> allNumbers = new ArrayList<>();
+      final int numNumbers = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 10));
+      for (int i = 0; i < numNumbers; ++i) {
+        allNumbers.add(random().nextLong());
+      }
+      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 Long number = allNumbers.get(random().nextInt(allNumbers.size()));
+        doc.add(new StringField("text", number.toString(), Field.Store.NO));
+        doc.add(new NumericDocValuesField("long", number));
+        doc.add(new SortedNumericDocValuesField("twolongs", number));
+        doc.add(new SortedNumericDocValuesField("twolongs", number * 2));
+        iw.addDocument(doc);
+      }
+      if (numNumbers > 1 && random().nextBoolean()) {
+        iw.deleteDocuments(new TermQuery(new Term("text", allNumbers.get(0).toString())));
+      }
+      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 numQueryNumbers =
+            TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 8));
+        Set<Long> queryNumbers = new HashSet<>();
+        Set<Long> queryNumbersX2 = new HashSet<>();
+        for (int j = 0; j < numQueryNumbers; ++j) {
+          Long number = allNumbers.get(random().nextInt(allNumbers.size()));
+          queryNumbers.add(number);
+          queryNumbersX2.add(2 * number);
+        }
+        long[] queryNumbersArray = queryNumbers.stream().mapToLong(Long::longValue).toArray();
+        long[] queryNumbersX2Array = queryNumbersX2.stream().mapToLong(Long::longValue).toArray();
+        final BooleanQuery.Builder bq = new BooleanQuery.Builder();
+        for (Long number : queryNumbers) {
+          bq.add(new TermQuery(new Term("text", number.toString())), BooleanClause.Occur.SHOULD);
+        }
+        Query q1 = new BoostQuery(new ConstantScoreQuery(bq.build()), boost);
+
+        Query q2 =
+            new BoostQuery(NumericDocValuesField.newSlowSetQuery("long", queryNumbersArray), boost);
+        assertSameMatches(searcher, q1, q2, true);
+
+        Query q3 =
+            new BoostQuery(
+                SortedNumericDocValuesField.newSlowSetQuery("twolongs", queryNumbersArray), boost);
+        assertSameMatches(searcher, q1, q3, true);
+
+        Query q4 =
+            new BoostQuery(
+                SortedNumericDocValuesField.newSlowSetQuery("twolongs", queryNumbersX2Array),
+                boost);
+        assertSameMatches(searcher, q1, q4, true);
+      }
+
+      reader.close();
+      dir.close();
+    }
+  }
 }
diff --git a/lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestDocValuesNumbersQuery.java b/lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestDocValuesNumbersQuery.java
deleted file mode 100644
index 0318ccb8ad9..00000000000
--- a/lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestDocValuesNumbersQuery.java
+++ /dev/null
@@ -1,203 +0,0 @@
-/*
- * 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.sandbox.search;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field.Store;
-import org.apache.lucene.document.NumericDocValuesField;
-import org.apache.lucene.document.SortedNumericDocValuesField;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.Term;
-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.Sort;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.tests.index.RandomIndexWriter;
-import org.apache.lucene.tests.util.LuceneTestCase;
-import org.apache.lucene.tests.util.TestUtil;
-import org.apache.lucene.util.IOUtils;
-
-public class TestDocValuesNumbersQuery extends LuceneTestCase {
-
-  public void testEquals() {
-    assertEquals(
-        new DocValuesNumbersQuery("field", 17L, 42L), new DocValuesNumbersQuery("field", 17L, 42L));
-    assertEquals(
-        new DocValuesNumbersQuery("field", 17L, 42L, 32416190071L),
-        new DocValuesNumbersQuery("field", 17L, 32416190071L, 42L));
-    assertFalse(
-        new DocValuesNumbersQuery("field", 42L).equals(new DocValuesNumbersQuery("field2", 42L)));
-    assertFalse(
-        new DocValuesNumbersQuery("field", 17L, 42L)
-            .equals(new DocValuesNumbersQuery("field", 17L, 32416190071L)));
-  }
-
-  public void testDuelTermsQuery() throws IOException {
-    final int iters = atLeast(2);
-    for (int iter = 0; iter < iters; ++iter) {
-      final List<Long> allNumbers = new ArrayList<>();
-      final int numNumbers = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 10));
-      for (int i = 0; i < numNumbers; ++i) {
-        allNumbers.add(random().nextLong());
-      }
-      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 Long number = allNumbers.get(random().nextInt(allNumbers.size()));
-        doc.add(new StringField("text", number.toString(), Store.NO));
-        doc.add(new NumericDocValuesField("long", number));
-        doc.add(new SortedNumericDocValuesField("twolongs", number));
-        doc.add(new SortedNumericDocValuesField("twolongs", number * 2));
-        iw.addDocument(doc);
-      }
-      if (numNumbers > 1 && random().nextBoolean()) {
-        iw.deleteDocuments(new TermQuery(new Term("text", allNumbers.get(0).toString())));
-      }
-      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 numQueryNumbers =
-            TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 8));
-        Set<Long> queryNumbers = new HashSet<>();
-        Set<Long> queryNumbersX2 = new HashSet<>();
-        for (int j = 0; j < numQueryNumbers; ++j) {
-          Long number = allNumbers.get(random().nextInt(allNumbers.size()));
-          queryNumbers.add(number);
-          queryNumbersX2.add(2 * number);
-        }
-        final BooleanQuery.Builder bq = new BooleanQuery.Builder();
-        for (Long number : queryNumbers) {
-          bq.add(new TermQuery(new Term("text", number.toString())), Occur.SHOULD);
-        }
-        Query q1 = new BoostQuery(new ConstantScoreQuery(bq.build()), boost);
-
-        Query q2 = new BoostQuery(new DocValuesNumbersQuery("long", queryNumbers), boost);
-        assertSameMatches(searcher, q1, q2, true);
-
-        Query q3 = new BoostQuery(new DocValuesNumbersQuery("twolongs", queryNumbers), boost);
-        assertSameMatches(searcher, q1, q3, true);
-
-        Query q4 = new BoostQuery(new DocValuesNumbersQuery("twolongs", queryNumbersX2), boost);
-        assertSameMatches(searcher, q1, q4, true);
-      }
-
-      reader.close();
-      dir.close();
-    }
-  }
-
-  public void testApproximation() throws IOException {
-    final int iters = atLeast(2);
-    for (int iter = 0; iter < iters; ++iter) {
-      final List<Long> allNumbers = new ArrayList<>();
-      final int numNumbers = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 10));
-      for (int i = 0; i < numNumbers; ++i) {
-        allNumbers.add(random().nextLong());
-      }
-      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 Long number = allNumbers.get(random().nextInt(allNumbers.size()));
-        doc.add(new StringField("text", number.toString(), Store.NO));
-        doc.add(new NumericDocValuesField("long", number));
-        iw.addDocument(doc);
-      }
-      if (numNumbers > 1 && random().nextBoolean()) {
-        iw.deleteDocuments(new TermQuery(new Term("text", allNumbers.get(0).toString())));
-      }
-      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 numQueryNumbers =
-            TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 8));
-        Set<Long> queryNumbers = new HashSet<>();
-        for (int j = 0; j < numQueryNumbers; ++j) {
-          queryNumbers.add(allNumbers.get(random().nextInt(allNumbers.size())));
-        }
-        final BooleanQuery.Builder bq = new BooleanQuery.Builder();
-        for (Long number : queryNumbers) {
-          bq.add(new TermQuery(new Term("text", number.toString())), Occur.SHOULD);
-        }
-        Query q1 = new BoostQuery(new ConstantScoreQuery(bq.build()), boost);
-        final Query q2 = new BoostQuery(new DocValuesNumbersQuery("long", queryNumbers), boost);
-
-        BooleanQuery.Builder bq1 = new BooleanQuery.Builder();
-        bq1.add(q1, Occur.MUST);
-        bq1.add(new TermQuery(new Term("text", allNumbers.get(0).toString())), Occur.FILTER);
-
-        BooleanQuery.Builder bq2 = new BooleanQuery.Builder();
-        bq2.add(q2, Occur.MUST);
-        bq2.add(new TermQuery(new Term("text", allNumbers.get(0).toString())), Occur.FILTER);
-
-        assertSameMatches(searcher, bq1.build(), bq2.build(), 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.value, td2.totalHits.value);
-    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);
-      }
-    }
-  }
-}