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

[09/50] [abbrv] lucene-solr:jira/http2: LUCENE-8340: Recency boosting.

LUCENE-8340: Recency boosting.


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

Branch: refs/heads/jira/http2
Commit: a9acdfdb544cb0f66709a89a9820e8444fe8edef
Parents: 34a8501
Author: Adrien Grand <jp...@gmail.com>
Authored: Tue Sep 4 11:48:25 2018 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Tue Sep 4 14:03:24 2018 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   6 +
 .../document/LongDistanceFeatureQuery.java      | 438 +++++++++++++++++++
 .../org/apache/lucene/document/LongPoint.java   |  26 ++
 .../search/BlockMaxConjunctionScorer.java       |   2 +-
 .../apache/lucene/search/ConjunctionScorer.java |   2 +-
 .../lucene/search/MaxScoreSumPropagator.java    |   2 +-
 .../org/apache/lucene/search/ReqExclScorer.java |   2 +-
 .../apache/lucene/search/ReqOptSumScorer.java   |   2 +-
 .../java/org/apache/lucene/search/Scorable.java |   2 +-
 .../search/ScoreCachingWrappingScorer.java      |   2 +-
 .../lucene/search/TopScoreDocCollector.java     |   2 +-
 .../org/apache/lucene/search/WANDScorer.java    |   2 +-
 .../document/TestLongDistanceFeatureQuery.java  | 350 +++++++++++++++
 .../apache/lucene/search/AssertingScorable.java |   2 +-
 .../apache/lucene/search/AssertingScorer.java   |   2 +-
 15 files changed, 831 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a9acdfdb/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index f95ce90..1334294 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -106,6 +106,12 @@ Changes in Runtime Behavior
   total hit counts accurately up to 1,000 in order to enable top-hits
   optimizations such as block-max WAND (LUCENE-8135). (Adrien Grand)
 
+New Features
+
+* LUCENE-8340: LongPoint#newDistanceQuery may be used to boost scores based on
+  how close a value of a long field is from an configurable origin. This is
+  typically useful to boost by recency. (Adrien Grand)
+
 Improvements
 
 * LUCENE-7997: Add BaseSimilarityTestCase to sanity check similarities.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a9acdfdb/lucene/core/src/java/org/apache/lucene/document/LongDistanceFeatureQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/LongDistanceFeatureQuery.java b/lucene/core/src/java/org/apache/lucene/document/LongDistanceFeatureQuery.java
new file mode 100644
index 0000000..480cfce
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/LongDistanceFeatureQuery.java
@@ -0,0 +1,438 @@
+/*
+ * 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.document;
+
+import java.io.IOException;
+import java.util.Objects;
+import java.util.Set;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.index.PointValues.IntersectVisitor;
+import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.ScorerSupplier;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.DocIdSetBuilder;
+import org.apache.lucene.util.FutureArrays;
+
+final class LongDistanceFeatureQuery extends Query {
+
+  private final String field;
+  private final long origin;
+  private final long pivotDistance;
+
+  LongDistanceFeatureQuery(String field, long origin, long pivotDistance) {
+    this.field = Objects.requireNonNull(field);
+    this.origin = origin;
+    if (pivotDistance <= 0) {
+      throw new IllegalArgumentException("pivotDistance must be > 0, got " + pivotDistance);
+    }
+    this.pivotDistance = pivotDistance;
+  }
+
+  @Override
+  public final boolean equals(Object o) {
+    return sameClassAs(o) &&
+        equalsTo(getClass().cast(o));
+  }
+
+  private boolean equalsTo(LongDistanceFeatureQuery other) {
+    return Objects.equals(field, other.field) &&
+        origin == other.origin &&
+        pivotDistance == other.pivotDistance;
+  }
+
+  @Override
+  public int hashCode() {
+    int h = classHash();
+    h = 31 * h + field.hashCode();
+    h = 31 * h + Long.hashCode(origin);
+    h = 31 * h + Long.hashCode(pivotDistance);
+    return h;
+  }
+
+  @Override
+  public String toString(String field) {
+    return getClass().getSimpleName() + "(field=" + field + ",origin=" + origin + ",pivotDistance=" + pivotDistance + ")";
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+    return new Weight(this) {
+
+      @Override
+      public boolean isCacheable(LeafReaderContext ctx) {
+        return false;
+      }
+
+      @Override
+      public void extractTerms(Set<Term> terms) {}
+
+      @Override
+      public Explanation explain(LeafReaderContext context, int doc) throws IOException {
+        SortedNumericDocValues multiDocValues = DocValues.getSortedNumeric(context.reader(), field);
+        if (multiDocValues.advanceExact(doc) == false) {
+          return Explanation.noMatch("Document " + doc + " doesn't have a value for field " + field);
+        }
+        long value = selectValue(multiDocValues);
+        long distance = Math.max(value, origin) - Math.min(value, origin);
+        if (distance < 0) {
+          // underflow, treat as MAX_VALUE
+          distance = Long.MAX_VALUE;
+        }
+        float score = (float) (boost * (pivotDistance / (pivotDistance + (double) distance)));
+        return Explanation.match(score, "Distance score, computed as weight * pivotDistance / (pivotDistance + abs(value - origin)) from:",
+            Explanation.match(boost, "weight"),
+            Explanation.match(pivotDistance, "pivotDistance"),
+            Explanation.match(origin, "origin"),
+            Explanation.match(value, "current value"));
+      }
+
+      private long selectValue(SortedNumericDocValues multiDocValues) throws IOException {
+        int count = multiDocValues.docValueCount();
+
+        long next = multiDocValues.nextValue();
+        if (count == 1 || next >= origin) {
+          return next;
+        }
+        long previous = next;
+        for (int i = 1; i < count; ++i) {
+          next = multiDocValues.nextValue();
+          if (next >= origin) {
+            // Unsigned comparison because of underflows
+            if (Long.compareUnsigned(origin - previous, next - origin) < 0) {
+              return previous;
+            } else {
+              return next;
+            }
+          }
+          previous = next;
+        }
+
+        assert next < origin;
+        return next;
+      }
+
+      private NumericDocValues selectValues(SortedNumericDocValues multiDocValues) {
+        final NumericDocValues singleton = DocValues.unwrapSingleton(multiDocValues);
+        if (singleton != null) {
+          return singleton;
+        }
+        return  new NumericDocValues() {
+
+          long value;
+
+          @Override
+          public long longValue() throws IOException {
+            return value;
+          }
+
+          @Override
+          public boolean advanceExact(int target) throws IOException {
+            if (multiDocValues.advanceExact(target)) {
+              value = selectValue(multiDocValues);
+              return true;
+            } else {
+              return false;
+            }
+          }
+
+          @Override
+          public int docID() {
+            return multiDocValues.docID();
+          }
+
+          @Override
+          public int nextDoc() throws IOException {
+            return multiDocValues.nextDoc();
+          }
+
+          @Override
+          public int advance(int target) throws IOException {
+            return multiDocValues.advance(target);
+          }
+
+          @Override
+          public long cost() {
+            return multiDocValues.cost();
+          }
+
+        };
+      }
+
+      @Override
+      public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
+        PointValues pointValues = context.reader().getPointValues(field);
+        if (pointValues == null) {
+          // No data on this segment
+          return null;
+        }
+        final SortedNumericDocValues multiDocValues = DocValues.getSortedNumeric(context.reader(), field);
+        final NumericDocValues docValues = selectValues(multiDocValues);
+
+        final Weight weight = this;
+        return new ScorerSupplier() {
+
+          @Override
+          public Scorer get(long leadCost) throws IOException {
+            return new DistanceScorer(weight, context.reader().maxDoc(), leadCost, boost, pointValues, docValues);
+          }
+
+          @Override
+          public long cost() {
+            return docValues.cost();
+          }
+        };
+      }
+
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        ScorerSupplier scorerSupplier = scorerSupplier(context);
+        if (scorerSupplier == null) {
+          return null;
+        }
+        return scorerSupplier.get(Long.MAX_VALUE);
+      }
+
+    };
+  }
+
+  private class DistanceScorer extends Scorer {
+
+    private final int maxDoc;
+    private DocIdSetIterator it;
+    private int doc = -1;
+    private final long leadCost;
+    private final float boost;
+    private final PointValues pointValues;
+    private final NumericDocValues docValues;
+    private long maxDistance = Long.MAX_VALUE;
+
+    protected DistanceScorer(Weight weight, int maxDoc, long leadCost, float boost,
+        PointValues pointValues, NumericDocValues docValues) {
+      super(weight);
+      this.maxDoc = maxDoc;
+      this.leadCost = leadCost;
+      this.boost = boost;
+      this.pointValues = pointValues;
+      this.docValues = docValues;
+      // initially use doc values in order to iterate all documents that have
+      // a value for this field
+      this.it = docValues;
+    }
+
+    @Override
+    public int docID() {
+      return doc;
+    }
+
+    private float score(double distance) {
+      return (float) (boost * (pivotDistance / (pivotDistance + distance)));
+    }
+
+    /**
+     * Inverting the score computation is very hard due to all potential
+     * rounding errors, so we binary search the maximum distance.
+     */
+    private long computeMaxDistance(float minScore, long previousMaxDistance) {
+      assert score(0) >= minScore;
+      if (score(previousMaxDistance) >= minScore) {
+        // minScore did not decrease enough to require an update to the max distance
+        return previousMaxDistance;
+      }
+      assert score(previousMaxDistance) < minScore;
+      long min = 0, max = previousMaxDistance;
+      // invariant: score(min) >= minScore && score(max) < minScore
+      while (max - min > 1) {
+        long mid = (min + max) >>> 1;
+        float score = score(mid);
+        if (score >= minScore) {
+          min = mid;
+        } else {
+          max = mid;
+        }
+      }
+      assert score(min) >= minScore;
+      assert min == Long.MAX_VALUE || score(min + 1) < minScore;
+      return min;
+    }
+
+    @Override
+    public float score() throws IOException {
+      if (docValues.advanceExact(docID()) == false) {
+        return 0;
+      }
+      long v = docValues.longValue();
+      // note: distance is unsigned
+      long distance = Math.max(v, origin) - Math.min(v, origin);
+      if (distance < 0) {
+        // underflow
+        // treat distances that are greater than MAX_VALUE as MAX_VALUE
+        distance = Long.MAX_VALUE;
+      }
+      return score(distance);
+    }
+
+    @Override
+    public DocIdSetIterator iterator() {
+      // add indirection so that if 'it' is updated then it will
+      // be taken into account
+      return new DocIdSetIterator() {
+
+        @Override
+        public int nextDoc() throws IOException {
+          return doc = it.nextDoc();
+        }
+
+        @Override
+        public int docID() {
+          return doc;
+        }
+
+        @Override
+        public long cost() {
+          return it.cost();
+        }
+
+        @Override
+        public int advance(int target) throws IOException {
+        return doc = it.advance(target);
+        }
+      };
+    }
+
+    @Override
+    public float getMaxScore(int upTo) {
+      return boost;
+    }
+
+    private int setMinCompetitiveScoreCounter = 0;
+
+    @Override
+    public void setMinCompetitiveScore(float minScore) throws IOException {
+      if (minScore > boost) {
+        it = DocIdSetIterator.empty();
+        return;
+      }
+
+      // Start sampling if we get called too much
+      setMinCompetitiveScoreCounter++;
+      if (setMinCompetitiveScoreCounter > 256 && (setMinCompetitiveScoreCounter & 0x1f) != 0x1f) {
+        return;
+      }
+
+      long previousMaxDistance = maxDistance;
+      maxDistance = computeMaxDistance(minScore, maxDistance);
+      if (maxDistance == previousMaxDistance) {
+        // nothing to update
+        return;
+      }
+      long minValue = origin - maxDistance;
+      if (minValue > origin) {
+        // underflow
+        minValue = Long.MIN_VALUE;
+      }
+      long maxValue = origin + maxDistance;
+      if (maxValue < origin) {
+        // overflow
+        maxValue = Long.MAX_VALUE;
+      }
+
+      final byte[] minValueAsBytes = new byte[Long.BYTES];
+      LongPoint.encodeDimension(minValue, minValueAsBytes, 0);
+      final byte[] maxValueAsBytes = new byte[Long.BYTES];
+      LongPoint.encodeDimension(maxValue, maxValueAsBytes, 0);
+
+      DocIdSetBuilder result = new DocIdSetBuilder(maxDoc);
+      final int doc = docID();
+      IntersectVisitor visitor = new IntersectVisitor() {
+
+        DocIdSetBuilder.BulkAdder adder;
+
+        @Override
+        public void grow(int count) {
+          adder = result.grow(count);
+        }
+
+        @Override
+        public void visit(int docID) {
+          if (docID <= doc) {
+            // Already visited or skipped
+            return;
+          }
+          adder.add(docID);
+        }
+
+        @Override
+        public void visit(int docID, byte[] packedValue) {
+          if (docID <= doc) {
+            // Already visited or skipped
+            return;
+          }
+          if (FutureArrays.compareUnsigned(packedValue, 0, Long.BYTES, minValueAsBytes, 0, Long.BYTES) < 0) {
+            // Doc's value is too low, in this dimension
+            return;
+          }
+          if (FutureArrays.compareUnsigned(packedValue, 0, Long.BYTES, maxValueAsBytes, 0, Long.BYTES) > 0) {
+            // Doc's value is too high, in this dimension
+            return;
+          }
+
+          // Doc is in-bounds
+          adder.add(docID);
+        }
+
+        @Override
+        public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+          if (FutureArrays.compareUnsigned(minPackedValue, 0, Long.BYTES, maxValueAsBytes, 0, Long.BYTES) > 0 ||
+              FutureArrays.compareUnsigned(maxPackedValue, 0, Long.BYTES, minValueAsBytes, 0, Long.BYTES) < 0) {
+            return Relation.CELL_OUTSIDE_QUERY;
+          }
+
+          if (FutureArrays.compareUnsigned(minPackedValue, 0, Long.BYTES, minValueAsBytes, 0, Long.BYTES) < 0 ||
+              FutureArrays.compareUnsigned(maxPackedValue, 0, Long.BYTES, maxValueAsBytes, 0, Long.BYTES) > 0) {
+            return Relation.CELL_CROSSES_QUERY;
+          }
+
+          return Relation.CELL_INSIDE_QUERY;
+        }
+      };
+
+      final long currentQueryCost = Math.min(leadCost, it.cost());
+      final long threshold = currentQueryCost >>> 3;
+      long estimatedNumberOfMatches = pointValues.estimatePointCount(visitor); // runs in O(log(numPoints))
+      // TODO: what is the right factor compared to the current disi? Is 8 optimal?
+      if (estimatedNumberOfMatches >= threshold) {
+        // the new range is not selective enough to be worth materializing
+        return;
+      }
+      pointValues.intersect(visitor);
+      it = result.build().iterator();
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a9acdfdb/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
index 686086c..5311114 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
@@ -20,9 +20,12 @@ import java.util.Arrays;
 import java.util.Collection;
 
 import org.apache.lucene.index.PointValues;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.BoostQuery;
 import org.apache.lucene.search.PointInSetQuery;
 import org.apache.lucene.search.PointRangeQuery;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.NumericUtils;
 
@@ -256,4 +259,27 @@ public final class LongPoint extends Field {
     }
     return newSetQuery(field, unboxed);
   }
+
+  /**
+   * Given a field that indexes the same long values into a {@link LongPoint}
+   * and doc values (either {@link NumericDocValuesField} or
+   * {@link SortedNumericDocValuesField}), this returns a query that scores
+   * documents based on their distance to {@code origin}:
+   * {@code score = weight * pivotDistance / (pivotDistance + distance)}, ie.
+   * score is in the {@code [0, weight]} range, is equal to {@code weight} when
+   * the document's value is equal to {@code origin} and is equal to
+   * {@code weight/2}  when the document's value is distant of
+   * {@code pivotDistance} from {@code origin}.
+   * In case of multi-valued fields, only the closest point to {@code origin}
+   * will be considered.
+   * This query is typically useful to boost results based on recency by adding
+   * this query to a {@link Occur#SHOULD} clause of a {@link BooleanQuery}.
+   */
+  public static Query newDistanceFeatureQuery(String field, float weight, long origin, long pivotDistance) {
+    Query query = new LongDistanceFeatureQuery(field, origin, pivotDistance);
+    if (weight != 1f) {
+      query = new BoostQuery(query, weight);
+    }
+    return query;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a9acdfdb/lucene/core/src/java/org/apache/lucene/search/BlockMaxConjunctionScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/BlockMaxConjunctionScorer.java b/lucene/core/src/java/org/apache/lucene/search/BlockMaxConjunctionScorer.java
index fa3743f..91fe76c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/BlockMaxConjunctionScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/BlockMaxConjunctionScorer.java
@@ -244,7 +244,7 @@ final class BlockMaxConjunctionScorer extends Scorer {
   }
 
   @Override
-  public void setMinCompetitiveScore(float score) {
+  public void setMinCompetitiveScore(float score) throws IOException {
     minScore = score;
     maxScorePropagator.setMinCompetitiveScore(score);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a9acdfdb/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 eafc57f..7ba4aa3 100644
--- a/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
@@ -83,7 +83,7 @@ class ConjunctionScorer extends Scorer {
   }
 
   @Override
-  public void setMinCompetitiveScore(float minScore) {
+  public void setMinCompetitiveScore(float minScore) throws IOException {
     // This scorer is only used for TOP_SCORES when there is a single scoring clause
     if (scorers.length == 1) {
       scorers[0].setMinCompetitiveScore(minScore);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a9acdfdb/lucene/core/src/java/org/apache/lucene/search/MaxScoreSumPropagator.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/MaxScoreSumPropagator.java b/lucene/core/src/java/org/apache/lucene/search/MaxScoreSumPropagator.java
index 331754c..1a4b3b5 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MaxScoreSumPropagator.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MaxScoreSumPropagator.java
@@ -110,7 +110,7 @@ final class MaxScoreSumPropagator {
     return scoreSumUpperBound(maxScore);
   }
 
-  void setMinCompetitiveScore(float minScore) {
+  void setMinCompetitiveScore(float minScore) throws IOException {
     if (minScore == 0) {
       return ;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a9acdfdb/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java b/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java
index d5c114a..90b11aa 100644
--- a/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java
@@ -87,7 +87,7 @@ class ReqExclScorer extends Scorer {
   }
 
   @Override
-  public void setMinCompetitiveScore(float score) {
+  public void setMinCompetitiveScore(float score) throws IOException {
     // The score of this scorer is the same as the score of 'reqScorer'.
     reqScorer.setMinCompetitiveScore(score);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a9acdfdb/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java b/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java
index 22df0af..fe9afc0 100644
--- a/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java
@@ -290,7 +290,7 @@ class ReqOptSumScorer extends Scorer {
   }
 
   @Override
-  public void setMinCompetitiveScore(float minScore) {
+  public void setMinCompetitiveScore(float minScore) throws IOException {
     this.minScore = minScore;
     // Potentially move to a conjunction
     if (reqMaxScore < minScore) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a9acdfdb/lucene/core/src/java/org/apache/lucene/search/Scorable.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/Scorable.java b/lucene/core/src/java/org/apache/lucene/search/Scorable.java
index 5102949..1fdda0d 100644
--- a/lucene/core/src/java/org/apache/lucene/search/Scorable.java
+++ b/lucene/core/src/java/org/apache/lucene/search/Scorable.java
@@ -45,7 +45,7 @@ public abstract class Scorable {
    * {@link ScoreMode#TOP_SCORES}, and successive calls may only set increasing
    * values of {@code minScore}.
    */
-  public void setMinCompetitiveScore(float minScore) {
+  public void setMinCompetitiveScore(float minScore) throws IOException {
     // no-op by default
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a9acdfdb/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java b/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java
index 0f11609..f68f25d 100644
--- a/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java
@@ -55,7 +55,7 @@ public final class ScoreCachingWrappingScorer extends Scorable {
   }
 
   @Override
-  public void setMinCompetitiveScore(float minScore) {
+  public void setMinCompetitiveScore(float minScore) throws IOException {
     in.setMinCompetitiveScore(minScore);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a9acdfdb/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
index 9a24406..eb8236c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
@@ -62,7 +62,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
       final int docBase = context.docBase;
       return new ScorerLeafCollector() {
 
-        private void updateMinCompetitiveScore() {
+        private void updateMinCompetitiveScore() throws IOException {
           // since we tie-break on doc id and collect in doc id order, we can require
           // the next float
           scorer.setMinCompetitiveScore(Math.nextUp(pqTop.score));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a9acdfdb/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java b/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java
index c6a5528..c9a6746 100644
--- a/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java
@@ -186,7 +186,7 @@ final class WANDScorer extends Scorer {
   }
 
   @Override
-  public void setMinCompetitiveScore(float minScore) {
+  public void setMinCompetitiveScore(float minScore) throws IOException {
     // Let this disjunction know about the new min score so that it can skip
     // over clauses that produce low scores.
     assert minScore >= 0;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a9acdfdb/lucene/core/src/test/org/apache/lucene/document/TestLongDistanceFeatureQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/document/TestLongDistanceFeatureQuery.java b/lucene/core/src/test/org/apache/lucene/document/TestLongDistanceFeatureQuery.java
new file mode 100644
index 0000000..d8ae66a
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/document/TestLongDistanceFeatureQuery.java
@@ -0,0 +1,350 @@
+/*
+ * 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.document;
+
+import java.io.IOException;
+
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.MultiReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.search.CheckHits;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryUtils;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TopScoreDocCollector;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestLongDistanceFeatureQuery extends LuceneTestCase {
+
+  public void testEqualsAndHashcode() {
+    Query q1 = LongPoint.newDistanceFeatureQuery("foo", 3, 10, 5);
+    Query q2 = LongPoint.newDistanceFeatureQuery("foo", 3, 10, 5);
+    QueryUtils.checkEqual(q1, q2);
+
+    Query q3 = LongPoint.newDistanceFeatureQuery("bar", 3, 10, 5);
+    QueryUtils.checkUnequal(q1, q3);
+
+    Query q4 = LongPoint.newDistanceFeatureQuery("foo", 4, 10, 5);
+    QueryUtils.checkUnequal(q1, q4);
+
+    Query q5 = LongPoint.newDistanceFeatureQuery("foo", 3, 9, 5);
+    QueryUtils.checkUnequal(q1, q5);
+
+    Query q6 = LongPoint.newDistanceFeatureQuery("foo", 3, 10, 6);
+    QueryUtils.checkUnequal(q1, q6);
+  }
+
+  public void testBasics() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, newIndexWriterConfig()
+        .setMergePolicy(newLogMergePolicy(random().nextBoolean())));
+    Document doc = new Document();
+    LongPoint point = new LongPoint("foo", 0L);
+    doc.add(point);
+    NumericDocValuesField docValue = new NumericDocValuesField("foo", 0L);
+    doc.add(docValue);
+
+    point.setLongValue(3);
+    docValue.setLongValue(3);
+    w.addDocument(doc);
+
+    point.setLongValue(12);
+    docValue.setLongValue(12);
+    w.addDocument(doc);
+
+    point.setLongValue(8);
+    docValue.setLongValue(8);
+    w.addDocument(doc);
+
+    point.setLongValue(-1);
+    docValue.setLongValue(-1);
+    w.addDocument(doc);
+
+    point.setLongValue(7);
+    docValue.setLongValue(7);
+    w.addDocument(doc);
+
+    DirectoryReader reader = w.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    
+    Query q = LongPoint.newDistanceFeatureQuery("foo", 3, 10, 5);
+    TopScoreDocCollector collector = TopScoreDocCollector.create(2, null, 1);
+    searcher.search(q, collector);
+    TopDocs topHits = collector.topDocs();
+    assertEquals(2, topHits.scoreDocs.length);
+
+    CheckHits.checkEqual(q,
+        new ScoreDoc[] {
+            new ScoreDoc(1, (float) (3f * (5. / (5. + 2.)))),
+            new ScoreDoc(2, (float) (3f * (5. / (5. + 2.))))
+        },
+        topHits.scoreDocs);
+
+    q = LongPoint.newDistanceFeatureQuery("foo", 3, 7, 5);
+    collector = TopScoreDocCollector.create(2, null, 1);
+    searcher.search(q, collector);
+    topHits = collector.topDocs();
+    assertEquals(2, topHits.scoreDocs.length);
+    CheckHits.checkExplanations(q, "", searcher);
+
+    CheckHits.checkEqual(q,
+        new ScoreDoc[] {
+            new ScoreDoc(4, (float) (3f * (5. / (5. + 0.)))),
+            new ScoreDoc(2, (float) (3f * (5. / (5. + 1.))))
+        },
+        topHits.scoreDocs);
+    
+    reader.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testOverUnderFlow() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, newIndexWriterConfig()
+        .setMergePolicy(newLogMergePolicy(random().nextBoolean())));
+    Document doc = new Document();
+    LongPoint point = new LongPoint("foo", 0L);
+    doc.add(point);
+    NumericDocValuesField docValue = new NumericDocValuesField("foo", 0L);
+    doc.add(docValue);
+
+    point.setLongValue(3);
+    docValue.setLongValue(3);
+    w.addDocument(doc);
+
+    point.setLongValue(12);
+    docValue.setLongValue(12);
+    w.addDocument(doc);
+
+    point.setLongValue(-10);
+    docValue.setLongValue(-10);
+    w.addDocument(doc);
+
+    point.setLongValue(Long.MAX_VALUE);
+    docValue.setLongValue(Long.MAX_VALUE);
+    w.addDocument(doc);
+
+    point.setLongValue(Long.MIN_VALUE);
+    docValue.setLongValue(Long.MIN_VALUE);
+    w.addDocument(doc);
+
+    DirectoryReader reader = w.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    
+    Query q = LongPoint.newDistanceFeatureQuery("foo", 3, Long.MAX_VALUE - 1, 100);
+    TopScoreDocCollector collector = TopScoreDocCollector.create(2, null, 1);
+    searcher.search(q, collector);
+    TopDocs topHits = collector.topDocs();
+    assertEquals(2, topHits.scoreDocs.length);
+
+    CheckHits.checkEqual(q,
+        new ScoreDoc[] {
+            new ScoreDoc(3, (float) (3f * (100. / (100. + 1.)))),
+            new ScoreDoc(0, (float) (3f * (100. / (100. + Long.MAX_VALUE)))) // rounding makes the distance treated as if it was MAX_VALUE
+        },
+        topHits.scoreDocs);
+
+    q = LongPoint.newDistanceFeatureQuery("foo", 3, Long.MIN_VALUE + 1, 100);
+    collector = TopScoreDocCollector.create(2, null, 1);
+    searcher.search(q, collector);
+    topHits = collector.topDocs();
+    assertEquals(2, topHits.scoreDocs.length);
+    CheckHits.checkExplanations(q, "", searcher);
+
+    CheckHits.checkEqual(q,
+        new ScoreDoc[] {
+            new ScoreDoc(4, (float) (3f * (100. / (100. + 1.)))),
+            new ScoreDoc(0, (float) (3f * (100. / (100. + Long.MAX_VALUE)))) // rounding makes the distance treated as if it was MAX_VALUE
+        },
+        topHits.scoreDocs);
+    
+    reader.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testMissingField() throws IOException {
+    IndexReader reader = new MultiReader();
+    IndexSearcher searcher = newSearcher(reader);
+    
+    Query q = LongPoint.newDistanceFeatureQuery("foo", 3, 10, 5);
+    TopDocs topHits = searcher.search(q, 2);
+    assertEquals(0, topHits.totalHits.value);
+  }
+
+  public void testMissingValue() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, newIndexWriterConfig()
+        .setMergePolicy(newLogMergePolicy(random().nextBoolean())));
+    Document doc = new Document();
+    LongPoint point = new LongPoint("foo", 0L);
+    doc.add(point);
+    NumericDocValuesField docValue = new NumericDocValuesField("foo", 0L);
+    doc.add(docValue);
+
+    point.setLongValue(3);
+    docValue.setLongValue(3);
+    w.addDocument(doc);
+
+    w.addDocument(new Document());
+
+    point.setLongValue(7);
+    docValue.setLongValue(7);
+    w.addDocument(doc);
+
+    DirectoryReader reader = w.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    
+    Query q = LongPoint.newDistanceFeatureQuery("foo", 3, 10, 5);
+    TopScoreDocCollector collector = TopScoreDocCollector.create(3, null, 1);
+    searcher.search(q, collector);
+    TopDocs topHits = collector.topDocs();
+    assertEquals(2, topHits.scoreDocs.length);
+
+    CheckHits.checkEqual(q,
+        new ScoreDoc[] {
+            new ScoreDoc(2, (float) (3f * (5. / (5. + 3.)))),
+            new ScoreDoc(0, (float) (3f * (5. / (5. + 7.))))
+        },
+        topHits.scoreDocs);
+
+    CheckHits.checkExplanations(q, "", searcher);
+
+    reader.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testMultiValued() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, newIndexWriterConfig()
+        .setMergePolicy(newLogMergePolicy(random().nextBoolean())));
+
+    Document doc = new Document();
+    for (long v : new long[] {3, 1000, Long.MAX_VALUE}) {
+      doc.add(new LongPoint("foo", v));
+      doc.add(new SortedNumericDocValuesField("foo", v));
+    }
+    w.addDocument(doc);
+
+    doc = new Document();
+    for (long v : new long[] {-100, 12, 999}) {
+      doc.add(new LongPoint("foo", v));
+      doc.add(new SortedNumericDocValuesField("foo", v));
+    }
+    w.addDocument(doc);
+
+    doc = new Document();
+    for (long v : new long[] {Long.MIN_VALUE, -1000, 8}) {
+      doc.add(new LongPoint("foo", v));
+      doc.add(new SortedNumericDocValuesField("foo", v));
+    }
+    w.addDocument(doc);
+
+    doc = new Document();
+    for (long v : new long[] { -1 }) {
+      doc.add(new LongPoint("foo", v));
+      doc.add(new SortedNumericDocValuesField("foo", v));
+    }
+    w.addDocument(doc);
+
+    doc = new Document();
+    for (long v : new long[] {Long.MIN_VALUE, 7}) {
+      doc.add(new LongPoint("foo", v));
+      doc.add(new SortedNumericDocValuesField("foo", v));
+    }
+    w.addDocument(doc);
+
+    DirectoryReader reader = w.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    
+    Query q = LongPoint.newDistanceFeatureQuery("foo", 3, 10, 5);
+    TopScoreDocCollector collector = TopScoreDocCollector.create(2, null, 1);
+    searcher.search(q, collector);
+    TopDocs topHits = collector.topDocs();
+    assertEquals(2, topHits.scoreDocs.length);
+
+    CheckHits.checkEqual(q,
+        new ScoreDoc[] {
+            new ScoreDoc(1, (float) (3f * (5. / (5. + 2.)))),
+            new ScoreDoc(2, (float) (3f * (5. / (5. + 2.))))
+        },
+        topHits.scoreDocs);
+
+    q = LongPoint.newDistanceFeatureQuery("foo", 3, 7, 5);
+    collector = TopScoreDocCollector.create(2, null, 1);
+    searcher.search(q, collector);
+    topHits = collector.topDocs();
+    assertEquals(2, topHits.scoreDocs.length);
+    CheckHits.checkExplanations(q, "", searcher);
+
+    CheckHits.checkEqual(q,
+        new ScoreDoc[] {
+            new ScoreDoc(4, (float) (3f * (5. / (5. + 0.)))),
+            new ScoreDoc(2, (float) (3f * (5. / (5. + 1.))))
+        },
+        topHits.scoreDocs);
+    
+    reader.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testRandom() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig()
+        .setMergePolicy(newLogMergePolicy(random().nextBoolean())));
+    Document doc = new Document();
+    LongPoint point = new LongPoint("foo", 0L);
+    doc.add(point);
+    NumericDocValuesField docValue = new NumericDocValuesField("foo", 0L);
+    doc.add(docValue);
+
+    int numDocs = atLeast(10000);
+    for (int i = 0; i < numDocs; ++i) {
+      long v = random().nextLong();
+      point.setLongValue(v);
+      docValue.setLongValue(v);
+      w.addDocument(doc);
+    }
+
+    IndexReader reader = DirectoryReader.open(w);
+    IndexSearcher searcher = newSearcher(reader);
+
+    for (int iter = 0; iter < 10; ++iter) {
+      long origin = random().nextLong();
+      long pivotDistance;
+      do {
+        pivotDistance = random().nextLong();
+      } while (pivotDistance <= 0);
+      float boost = (1 + random().nextInt(10)) / 3f;
+      Query q = LongPoint.newDistanceFeatureQuery("foo", boost, origin, pivotDistance);
+
+      CheckHits.checkTopScores(random(), q, searcher);
+    }
+
+    reader.close();
+    w.close();
+    dir.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a9acdfdb/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorable.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorable.java b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorable.java
index f69fe60..208eb4b 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorable.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorable.java
@@ -39,7 +39,7 @@ public class AssertingScorable extends FilterScorable {
   }
 
   @Override
-  public void setMinCompetitiveScore(float minScore) {
+  public void setMinCompetitiveScore(float minScore) throws IOException {
     in.setMinCompetitiveScore(minScore);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a9acdfdb/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorer.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorer.java b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorer.java
index 0319ce9..01477f3 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorer.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorer.java
@@ -66,7 +66,7 @@ public class AssertingScorer extends Scorer {
   }
 
   @Override
-  public void setMinCompetitiveScore(float score) {
+  public void setMinCompetitiveScore(float score) throws IOException {
     assert scoreMode == ScoreMode.TOP_SCORES;
     assert Float.isNaN(score) == false;
     assert score >= minCompetitiveScore;