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

[10/45] lucene-solr:jira/solr-12709: LUCENE-8482: Added feature query in LatLonPoint to boost results by distance

LUCENE-8482: Added feature query in LatLonPoint to boost results by distance


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

Branch: refs/heads/jira/solr-12709
Commit: 10060a6237ccd2785f6cbe248ca7254028f8eb04
Parents: c06a7ab
Author: iverase <iv...@apache.org>
Authored: Wed Sep 19 10:49:56 2018 +0200
Committer: iverase <iv...@apache.org>
Committed: Wed Sep 19 10:49:56 2018 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   4 +
 .../org/apache/lucene/document/LatLonPoint.java |  24 +
 .../LatLonPointDistanceFeatureQuery.java        | 480 +++++++++++++++++++
 .../TestLatLonPointDistanceFeatureQuery.java    | 411 ++++++++++++++++
 4 files changed, 919 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/10060a62/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 96c1c19..98631f3 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -117,6 +117,10 @@ New Features
   how close a value of a long field is from an configurable origin. This is
   typically useful to boost by recency. (Adrien Grand)
 
+* LUCENE-8482: LatLonPoint#newDistanceFeatureQuery may be used to boost scores
+  based on the haversine distance of a LatLonPoint field to a provided point. This is
+  typically useful to boost by distance. (Ignacio Vera)
+
 Improvements
 
 * LUCENE-7997: Add BaseSimilarityTestCase to sanity check similarities.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/10060a62/lucene/core/src/java/org/apache/lucene/document/LatLonPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/LatLonPoint.java b/lucene/core/src/java/org/apache/lucene/document/LatLonPoint.java
index 6cf8ab5..e3edead 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LatLonPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LatLonPoint.java
@@ -21,6 +21,8 @@ import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.PointValues;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BoostQuery;
 import org.apache.lucene.search.ConstantScoreQuery;
 import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.search.PointRangeQuery;
@@ -252,4 +254,26 @@ public class LatLonPoint extends Field {
     return new LatLonPointInPolygonQuery(field, polygons);
   }
 
+  /**
+   * Given a field that indexes point values into a {@link LatLonPoint}
+   * and doc values into {@link LatLonDocValuesField}, this returns a query that scores
+   * documents based on their haversine distance in meters to {@code (originLat, originLon)}:
+   * {@code score = weight * pivotDistanceMeters / (pivotDistanceMeters + distance)}, ie.
+   * score is in the {@code [0, weight]} range, is equal to {@code weight} when
+   * the document's value is equal to {@code (originLat, originLon)} and is equal to
+   * {@code weight/2}  when the document's value is distant of
+   * {@code pivotDistanceMeters} from {@code (originLat, originLon)}.
+   * In case of multi-valued fields, only the closest point to {@code (originLat, originLon)}
+   * will be considered.
+   * This query is typically useful to boost results based on distance by adding
+   * this query to a {@link Occur#SHOULD} clause of a {@link BooleanQuery}.
+   */
+  public static Query newDistanceFeatureQuery(String field, float weight, double originLat, double originLon, double pivotDistanceMeters) {
+    Query query = new LatLonPointDistanceFeatureQuery(field, originLat, originLon, pivotDistanceMeters);
+    if (weight != 1f) {
+      query = new BoostQuery(query, weight);
+    }
+    return query;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/10060a62/lucene/core/src/java/org/apache/lucene/document/LatLonPointDistanceFeatureQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/LatLonPointDistanceFeatureQuery.java b/lucene/core/src/java/org/apache/lucene/document/LatLonPointDistanceFeatureQuery.java
new file mode 100644
index 0000000..3f30dea
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/LatLonPointDistanceFeatureQuery.java
@@ -0,0 +1,480 @@
+/*
+ * 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.geo.GeoEncodingUtils;
+import org.apache.lucene.geo.GeoUtils;
+import org.apache.lucene.geo.Rectangle;
+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;
+import org.apache.lucene.util.NumericUtils;
+import org.apache.lucene.util.SloppyMath;
+
+
+final class LatLonPointDistanceFeatureQuery extends Query {
+
+  private final String field;
+  private final double originLat;
+  private final double originLon;
+  private final double pivotDistance;
+
+  LatLonPointDistanceFeatureQuery(String field, double originLat, double originLon, double pivotDistance) {
+    this.field = Objects.requireNonNull(field);
+    GeoUtils.checkLatitude(originLat);
+    GeoUtils.checkLongitude(originLon);
+    this.originLon = originLon;
+    this.originLat = originLat;
+    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(LatLonPointDistanceFeatureQuery other) {
+    return Objects.equals(field, other.field) &&
+        originLon == other.originLon &&
+        originLat == other.originLat &&
+        pivotDistance == other.pivotDistance;
+  }
+
+  @Override
+  public int hashCode() {
+    int h = classHash();
+    h = 31 * h + field.hashCode();
+    h = 31 * h + Double.hashCode(originLat);
+    h = 31 * h + Double.hashCode(originLon);
+    h = 31 * h + Double.hashCode(pivotDistance);
+    return h;
+  }
+
+  @Override
+  public String toString(String field) {
+    return getClass().getSimpleName() + "(field=" + field + ",originLat=" + originLat + ",originLon=" + originLon + ",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 encoded = selectValue(multiDocValues);
+        int latitudeBits = (int)(encoded >> 32);
+        int longitudeBits = (int)(encoded & 0xFFFFFFFF);
+        double lat = GeoEncodingUtils.decodeLatitude(latitudeBits);
+        double lon = GeoEncodingUtils.decodeLongitude(longitudeBits);
+        double distance = SloppyMath.haversinMeters(originLat, originLon, lat, lon);
+        float score = (float) (boost * (pivotDistance / (pivotDistance + distance)));
+        return Explanation.match(score, "Distance score, computed as weight * pivotDistance / (pivotDistance + abs(distance)) from:",
+            Explanation.match(boost, "weight"),
+            Explanation.match(pivotDistance, "pivotDistance"),
+            Explanation.match(originLat, "originLat"),
+            Explanation.match(originLon, "originLon"),
+            Explanation.match(lat, "current lat"),
+            Explanation.match(lon, "current lon"),
+            Explanation.match(distance, "distance"));
+      }
+
+      private long selectValue(SortedNumericDocValues multiDocValues) throws IOException {
+        int count = multiDocValues.docValueCount();
+        long value = multiDocValues.nextValue();
+        if (count == 1) {
+          return value;
+        }
+        // compute exact sort key: avoid any asin() computations
+        double distance = getDistanceKeyFromEncoded(value);
+        for (int i = 1; i < count; ++i) {
+          long nextValue = multiDocValues.nextValue();
+          double nextDistance = getDistanceKeyFromEncoded(nextValue);
+          if (nextDistance < distance) {
+            distance = nextDistance;
+            value = nextValue;
+          }
+        }
+        return value;
+      }
+
+      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 double getDistanceFromEncoded(long encoded) {
+    return SloppyMath.haversinMeters(getDistanceKeyFromEncoded(encoded));
+  }
+
+  private double getDistanceKeyFromEncoded(long encoded) {
+    int latitudeBits = (int)(encoded >> 32);
+    int longitudeBits = (int)(encoded & 0xFFFFFFFF);
+    double lat = GeoEncodingUtils.decodeLatitude(latitudeBits);
+    double lon = GeoEncodingUtils.decodeLongitude(longitudeBits);
+    return SloppyMath.haversinSortKey(originLat, originLon, lat, lon);
+  }
+
+  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 double maxDistance = GeoUtils.EARTH_MEAN_RADIUS_METERS * Math.PI;
+
+    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. The limit
+     * is set to 1 meter.
+     */
+    private double computeMaxDistance(float minScore, double 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;
+      double min = 0, max = previousMaxDistance;
+      // invariant: score(min) >= minScore && score(max) < minScore
+      while (max - min > 1) {
+        double mid = (min + max) / 2;
+        float score = score(mid);
+        if (score >= minScore) {
+          min = mid;
+        } else {
+          max = mid;
+        }
+      }
+      assert score(min) >= minScore;
+      assert min == Double.MAX_VALUE || score(min + 1) < minScore;
+      return min;
+    }
+
+    @Override
+    public float score() throws IOException {
+      if (docValues.advanceExact(docID()) == false) {
+        return 0;
+      }
+      return score(getDistanceFromEncoded(docValues.longValue()));
+    }
+
+    @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;
+      }
+
+      setMinCompetitiveScoreCounter++;
+      // We sample the calls to this method as it is expensive to recalculate the iterator.
+      if (setMinCompetitiveScoreCounter > 256 && (setMinCompetitiveScoreCounter & 0x1f) != 0x1f) {
+        return;
+      }
+
+      double previousMaxDistance = maxDistance;
+      maxDistance = computeMaxDistance(minScore, maxDistance);
+      if (maxDistance == previousMaxDistance) {
+        // nothing to update
+        return;
+      }
+
+      //Ideally we would be doing a distance query but that is too expensive so we approximate
+      //with a box query which performs better.
+      Rectangle box = Rectangle.fromPointDistance(originLat, originLon, maxDistance);
+      final byte minLat[] = new byte[LatLonPoint.BYTES];
+      final byte maxLat[] = new byte[LatLonPoint.BYTES];
+      final byte minLon[] = new byte[LatLonPoint.BYTES];
+      final byte maxLon[] = new byte[LatLonPoint.BYTES];
+      final boolean crossDateLine = box.crossesDateline();
+
+
+      NumericUtils.intToSortableBytes(GeoEncodingUtils.encodeLatitude(box.minLat), minLat, 0);
+      NumericUtils.intToSortableBytes(GeoEncodingUtils.encodeLatitude(box.maxLat), maxLat, 0);
+      NumericUtils.intToSortableBytes(GeoEncodingUtils.encodeLongitude(box.minLon), minLon, 0);
+      NumericUtils.intToSortableBytes(GeoEncodingUtils.encodeLongitude(box.maxLon), maxLon, 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, LatLonPoint.BYTES, maxLat, 0, LatLonPoint.BYTES) > 0 ||
+              FutureArrays.compareUnsigned(packedValue, 0, LatLonPoint.BYTES, minLat, 0, LatLonPoint.BYTES) < 0) {
+            //Latitude out of range
+            return;
+          }
+          if (crossDateLine) {
+            if (FutureArrays.compareUnsigned(packedValue, LatLonPoint.BYTES, 2 * LatLonPoint.BYTES, minLon, 0, LatLonPoint.BYTES) < 0 &&
+                FutureArrays.compareUnsigned(packedValue, LatLonPoint.BYTES, 2 * LatLonPoint.BYTES, maxLon, 0, LatLonPoint.BYTES)  > 0) {
+              //Longitude out of range
+              return;
+            }
+
+          } else {
+            if (FutureArrays.compareUnsigned(packedValue, LatLonPoint.BYTES, 2 * LatLonPoint.BYTES, maxLon, 0, LatLonPoint.BYTES) > 0 ||
+                FutureArrays.compareUnsigned(packedValue, LatLonPoint.BYTES, 2 * LatLonPoint.BYTES, minLon, 0, LatLonPoint.BYTES) < 0) {
+              //Longitude out of range
+              return;
+            }
+          }
+          adder.add(docID);
+        }
+
+        @Override
+        public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+
+          if (FutureArrays.compareUnsigned(minPackedValue, 0, LatLonPoint.BYTES, maxLat, 0, LatLonPoint.BYTES) > 0 ||
+              FutureArrays.compareUnsigned(maxPackedValue, 0, LatLonPoint.BYTES, minLat, 0, LatLonPoint.BYTES) < 0) {
+            return Relation.CELL_OUTSIDE_QUERY;
+          }
+          boolean crosses = FutureArrays.compareUnsigned(minPackedValue, 0, LatLonPoint.BYTES, minLat, 0, LatLonPoint.BYTES) < 0 ||
+              FutureArrays.compareUnsigned(maxPackedValue, 0, LatLonPoint.BYTES, maxLat, 0, LatLonPoint.BYTES) > 0;
+
+          if (crossDateLine) {
+            if (FutureArrays.compareUnsigned(minPackedValue, LatLonPoint.BYTES, 2 * LatLonPoint.BYTES, maxLon, 0, LatLonPoint.BYTES) > 0 &&
+                FutureArrays.compareUnsigned(maxPackedValue, LatLonPoint.BYTES, 2 * LatLonPoint.BYTES, minLon, 0, LatLonPoint.BYTES) < 0) {
+              return Relation.CELL_OUTSIDE_QUERY;
+            }
+            crosses |= FutureArrays.compareUnsigned(minPackedValue, LatLonPoint.BYTES, 2 * LatLonPoint.BYTES, maxLon, 0, LatLonPoint.BYTES) < 0 ||
+                FutureArrays.compareUnsigned(maxPackedValue, LatLonPoint.BYTES, 2 * LatLonPoint.BYTES, minLon, 0, LatLonPoint.BYTES) > 0;
+
+          } else {
+            if (FutureArrays.compareUnsigned(minPackedValue, LatLonPoint.BYTES, 2 * LatLonPoint.BYTES, maxLon, 0, LatLonPoint.BYTES) > 0 ||
+                FutureArrays.compareUnsigned(maxPackedValue, LatLonPoint.BYTES, 2 * LatLonPoint.BYTES, minLon, 0, LatLonPoint.BYTES) < 0) {
+              return Relation.CELL_OUTSIDE_QUERY;
+            }
+            crosses |= FutureArrays.compareUnsigned(minPackedValue, LatLonPoint.BYTES, 2 * LatLonPoint.BYTES, minLon, 0, LatLonPoint.BYTES) < 0 ||
+                FutureArrays.compareUnsigned(maxPackedValue, LatLonPoint.BYTES, 2 * LatLonPoint.BYTES, maxLon, 0, LatLonPoint.BYTES) > 0;
+          }
+          if (crosses) {
+            return Relation.CELL_CROSSES_QUERY;
+          } else {
+            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/10060a62/lucene/core/src/test/org/apache/lucene/document/TestLatLonPointDistanceFeatureQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/document/TestLatLonPointDistanceFeatureQuery.java b/lucene/core/src/test/org/apache/lucene/document/TestLatLonPointDistanceFeatureQuery.java
new file mode 100644
index 0000000..2010bc5
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/document/TestLatLonPointDistanceFeatureQuery.java
@@ -0,0 +1,411 @@
+/*
+ * 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.geo.GeoEncodingUtils;
+import org.apache.lucene.geo.GeoUtils;
+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.MatchAllDocsQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryUtils;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TopScoreDocCollector;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.SloppyMath;
+
+public class TestLatLonPointDistanceFeatureQuery extends LuceneTestCase {
+
+  public void testEqualsAndHashcode() {
+    Query q1 = LatLonPoint.newDistanceFeatureQuery("foo", 3, 10, 10, 5);
+    Query q2 = LatLonPoint.newDistanceFeatureQuery("foo", 3, 10, 10, 5);
+    QueryUtils.checkEqual(q1, q2);
+
+    Query q3 = LatLonPoint.newDistanceFeatureQuery("bar", 3, 10, 10,5);
+    QueryUtils.checkUnequal(q1, q3);
+
+    Query q4 = LatLonPoint.newDistanceFeatureQuery("foo", 4, 10, 10, 5);
+    QueryUtils.checkUnequal(q1, q4);
+
+    Query q5 = LatLonPoint.newDistanceFeatureQuery("foo", 3, 9, 10, 5);
+    QueryUtils.checkUnequal(q1, q5);
+
+    Query q6 = LatLonPoint.newDistanceFeatureQuery("foo", 3, 10, 9, 5);
+    QueryUtils.checkUnequal(q1, q6);
+
+    Query q7 = LatLonPoint.newDistanceFeatureQuery("foo", 3, 10, 10, 6);
+    QueryUtils.checkUnequal(q1, q7);
+  }
+
+  public void testBasics() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, newIndexWriterConfig()
+        .setMergePolicy(newLogMergePolicy(random().nextBoolean())));
+    Document doc = new Document();
+    LatLonPoint point = new LatLonPoint("foo", 0.0, 0.0);
+    doc.add(point);
+    LatLonDocValuesField docValue = new LatLonDocValuesField("foo",0.0, 0.0);
+    doc.add(docValue);
+
+    double pivotDistance = 5000;//5k
+
+    point.setLocationValue(-7, -7);
+    docValue.setLocationValue(-7, -7);
+    w.addDocument(doc);
+
+    point.setLocationValue(9, 9);
+    docValue.setLocationValue(9, 9);
+    w.addDocument(doc);
+
+
+    point.setLocationValue(8, 8);
+    docValue.setLocationValue(8, 8);
+    w.addDocument(doc);
+
+    point.setLocationValue(4, 4);
+    docValue.setLocationValue(4, 4);
+    w.addDocument(doc);
+
+    point.setLocationValue(-1, -1);
+    docValue.setLocationValue(-1, -1);
+    w.addDocument(doc);
+
+    DirectoryReader reader = w.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    
+    Query q = LatLonPoint.newDistanceFeatureQuery("foo", 3, 10, 10, pivotDistance);
+    TopScoreDocCollector collector = TopScoreDocCollector.create(2, null, 1);
+    searcher.search(q, collector);
+    TopDocs topHits = collector.topDocs();
+    assertEquals(2, topHits.scoreDocs.length);
+
+    double distance1 = SloppyMath.haversinMeters(GeoEncodingUtils.decodeLatitude(GeoEncodingUtils.encodeLatitude(9)) , GeoEncodingUtils.decodeLongitude(GeoEncodingUtils.encodeLongitude(9)), 10,10);
+    double distance2 = SloppyMath.haversinMeters(GeoEncodingUtils.decodeLatitude(GeoEncodingUtils.encodeLatitude(8)) , GeoEncodingUtils.decodeLongitude(GeoEncodingUtils.encodeLongitude(8)), 10,10);
+
+    CheckHits.checkEqual(q,
+        new ScoreDoc[] {
+            new ScoreDoc(1, (float) (3f * (pivotDistance / (pivotDistance + distance1)))),
+            new ScoreDoc(2, (float) (3f * (pivotDistance / (pivotDistance + distance2))))
+        },
+        topHits.scoreDocs);
+
+    distance1 = SloppyMath.haversinMeters(GeoEncodingUtils.decodeLatitude(GeoEncodingUtils.encodeLatitude(9)) , GeoEncodingUtils.decodeLongitude(GeoEncodingUtils.encodeLongitude(9)), 9,9);
+    distance2 = SloppyMath.haversinMeters(GeoEncodingUtils.decodeLatitude(GeoEncodingUtils.encodeLatitude(8)) , GeoEncodingUtils.decodeLongitude(GeoEncodingUtils.encodeLongitude(8)), 9,9);
+
+    q = LatLonPoint.newDistanceFeatureQuery("foo", 3, 9, 9,  pivotDistance);
+    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(1, (float) (3f * (pivotDistance / (pivotDistance + distance1)))),
+            new ScoreDoc(2, (float) (3f * (pivotDistance / (pivotDistance + distance2))))
+        },
+        topHits.scoreDocs);
+    
+    reader.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testCrossesDateLine() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, newIndexWriterConfig()
+        .setMergePolicy(newLogMergePolicy(random().nextBoolean())));
+    Document doc = new Document();
+    LatLonPoint point = new LatLonPoint("foo", 0.0, 0.0);
+    doc.add(point);
+    LatLonDocValuesField docValue = new LatLonDocValuesField("foo",0.0, 0.0);
+    doc.add(docValue);
+
+    double pivotDistance = 5000;//5k
+
+    point.setLocationValue(0, -179);
+    docValue.setLocationValue(0, -179);
+    w.addDocument(doc);
+
+    point.setLocationValue(0, 176);
+    docValue.setLocationValue(0, 176);
+    w.addDocument(doc);
+
+    point.setLocationValue(0, -150);
+    docValue.setLocationValue(0, -150);
+    w.addDocument(doc);
+
+    point.setLocationValue(0, -140);
+    docValue.setLocationValue(0, -140);
+    w.addDocument(doc);
+
+    point.setLocationValue(0, 140);
+    docValue.setLocationValue(01, 140);
+    w.addDocument(doc);
+
+    DirectoryReader reader = w.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+
+    Query q = LatLonPoint.newDistanceFeatureQuery("foo", 3, 0, 179, pivotDistance);
+    TopScoreDocCollector collector = TopScoreDocCollector.create(2, null, 1);
+    searcher.search(q, collector);
+    TopDocs topHits = collector.topDocs();
+    assertEquals(2, topHits.scoreDocs.length);
+
+    double distance1 = SloppyMath.haversinMeters(GeoEncodingUtils.decodeLatitude(GeoEncodingUtils.encodeLatitude(0)) , GeoEncodingUtils.decodeLongitude(GeoEncodingUtils.encodeLongitude(-179)), 0,179);
+    double distance2 = SloppyMath.haversinMeters(GeoEncodingUtils.decodeLatitude(GeoEncodingUtils.encodeLatitude(0)) , GeoEncodingUtils.decodeLongitude(GeoEncodingUtils.encodeLongitude(176)), 0,179);
+
+    CheckHits.checkEqual(q,
+        new ScoreDoc[] {
+            new ScoreDoc(0, (float) (3f * (pivotDistance / (pivotDistance + distance1)))),
+            new ScoreDoc(1, (float) (3f * (pivotDistance / (pivotDistance + distance2))))
+        },
+        topHits.scoreDocs);
+
+    reader.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testMissingField() throws IOException {
+    IndexReader reader = new MultiReader();
+    IndexSearcher searcher = newSearcher(reader);
+    
+    Query q = LatLonPoint.newDistanceFeatureQuery("foo", 3, 10, 10, 5000);
+    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();
+    LatLonPoint point = new LatLonPoint("foo", 0, 0);
+    doc.add(point);
+    LatLonDocValuesField docValue = new LatLonDocValuesField("foo", 0, 0);
+    doc.add(docValue);
+
+    point.setLocationValue(3, 3);
+    docValue.setLocationValue(3, 3);
+    w.addDocument(doc);
+
+    w.addDocument(new Document());
+
+    point.setLocationValue(7, 7);
+    docValue.setLocationValue(7, 7);
+    w.addDocument(doc);
+
+    DirectoryReader reader = w.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    
+    Query q = LatLonPoint.newDistanceFeatureQuery("foo", 3, 10, 10, 5);
+    TopScoreDocCollector collector = TopScoreDocCollector.create(3, null, 1);
+    searcher.search(q, collector);
+    TopDocs topHits = collector.topDocs();
+    assertEquals(2, topHits.scoreDocs.length);
+
+    double distance1 = SloppyMath.haversinMeters(GeoEncodingUtils.decodeLatitude(GeoEncodingUtils.encodeLatitude(7)) , GeoEncodingUtils.decodeLongitude(GeoEncodingUtils.encodeLongitude(7)), 10,10);
+    double distance2 = SloppyMath.haversinMeters(GeoEncodingUtils.decodeLatitude(GeoEncodingUtils.encodeLatitude(3)) , GeoEncodingUtils.decodeLongitude(GeoEncodingUtils.encodeLongitude(3)), 10,10);
+
+    CheckHits.checkEqual(q,
+        new ScoreDoc[] {
+            new ScoreDoc(2, (float) (3f * (5. / (5. + distance1)))),
+            new ScoreDoc(0, (float) (3f * (5. / (5. + distance2))))
+        },
+        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 (double[] point  : new double[][] {{0, 0}, {30, 30}, {60, 60}}) {
+      doc.add(new LatLonPoint("foo", point[0], point[1]));
+      doc.add(new LatLonDocValuesField("foo", point[0], point[1]));
+    }
+    w.addDocument(doc);
+
+    doc = new Document();
+    for (double[] point  : new double[][] {{45, 0}, {-45, 0}, {-90, 0}, {90, 0}}) {
+      doc.add(new LatLonPoint("foo", point[0], point[1]));
+      doc.add(new LatLonDocValuesField("foo", point[0], point[1]));
+    }
+    w.addDocument(doc);
+
+    doc = new Document();
+    for (double[] point  : new double[][] {{0, 90}, {0, -90}, {0, 180}, {0, -180}}) {
+      doc.add(new LatLonPoint("foo", point[0], point[1]));
+      doc.add(new LatLonDocValuesField("foo", point[0], point[1]));
+    }
+    w.addDocument(doc);
+
+    doc = new Document();
+    for (double[] point  : new double[][] {{3, 2}}) {
+      doc.add(new LatLonPoint("foo", point[0], point[1]));
+      doc.add(new LatLonDocValuesField("foo", point[0], point[1]));
+    }
+    w.addDocument(doc);
+
+    doc = new Document();
+    for (double[] point  : new double[][] {{45, 45}, {-45, -45}}) {
+      doc.add(new LatLonPoint("foo", point[0], point[1]));
+      doc.add(new LatLonDocValuesField("foo", point[0], point[1]));
+    }
+    w.addDocument(doc);
+
+    DirectoryReader reader = w.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+
+    Query q = LatLonPoint.newDistanceFeatureQuery("foo", 3, 0, 0, 200);
+    TopScoreDocCollector collector = TopScoreDocCollector.create(2, null, 1);
+    searcher.search(q, collector);
+    TopDocs topHits = collector.topDocs();
+    assertEquals(2, topHits.scoreDocs.length);
+
+    double distance1 = SloppyMath.haversinMeters(GeoEncodingUtils.decodeLatitude(GeoEncodingUtils.encodeLatitude(0)) , GeoEncodingUtils.decodeLongitude(GeoEncodingUtils.encodeLongitude(0)), 0,0);
+    double distance2 = SloppyMath.haversinMeters(GeoEncodingUtils.decodeLatitude(GeoEncodingUtils.encodeLatitude(3)) , GeoEncodingUtils.decodeLongitude(GeoEncodingUtils.encodeLongitude(2)), 0,0);
+
+    CheckHits.checkEqual(q,
+        new ScoreDoc[] {
+            new ScoreDoc(0, (float) (3f * (200 / (200 + distance1)))),
+            new ScoreDoc(3, (float) (3f * (200 / (200 + distance2))))
+        },
+        topHits.scoreDocs);
+
+    q = LatLonPoint.newDistanceFeatureQuery("foo", 3, -90, 0, 10000.);
+    collector = TopScoreDocCollector.create(2, null, 1);
+    searcher.search(q, collector);
+    topHits = collector.topDocs();
+    assertEquals(2, topHits.scoreDocs.length);
+    CheckHits.checkExplanations(q, "", searcher);
+
+    distance1 = SloppyMath.haversinMeters(GeoEncodingUtils.decodeLatitude(GeoEncodingUtils.encodeLatitude(-90)) , GeoEncodingUtils.decodeLongitude(GeoEncodingUtils.encodeLongitude(0)), -90,0);
+    distance2 = SloppyMath.haversinMeters(GeoEncodingUtils.decodeLatitude(GeoEncodingUtils.encodeLatitude(-45)) , GeoEncodingUtils.decodeLongitude(GeoEncodingUtils.encodeLongitude(-45)), -90,0);
+
+    CheckHits.checkEqual(q,
+        new ScoreDoc[] {
+            new ScoreDoc(1, (float) (3f * (10000. / (10000. + distance1)))),
+            new ScoreDoc(4, (float) (3f * (10000. / (10000. + distance2))))
+        },
+        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();
+    LatLonPoint point = new LatLonPoint("foo", 0., 0.);
+    doc.add(point);
+    LatLonDocValuesField docValue = new LatLonDocValuesField("foo", 0., 0.);
+    doc.add(docValue);
+
+    int numDocs = atLeast(10000);
+    for (int i = 0; i < numDocs; ++i) {
+      double lat = random().nextDouble() * 180 - 90;
+      double lon = random().nextDouble() * 360 - 180;
+      point.setLocationValue(lat, lon);
+      docValue.setLocationValue(lat, lon);
+      w.addDocument(doc);
+    }
+
+    IndexReader reader = DirectoryReader.open(w);
+    IndexSearcher searcher = newSearcher(reader);
+
+    for (int iter = 0; iter < 10; ++iter) {
+      double lat = random().nextDouble() * 180 - 90;
+      double lon = random().nextDouble() * 360 - 180;
+      double  pivotDistance = random().nextDouble() * random().nextDouble() * Math.PI * GeoUtils.EARTH_MEAN_RADIUS_METERS;
+      float boost = (1 + random().nextInt(10)) / 3f;
+      Query q = LatLonPoint.newDistanceFeatureQuery("foo", boost, lat, lon, pivotDistance);
+
+      CheckHits.checkTopScores(random(), q, searcher);
+    }
+
+    reader.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testCompareSorting() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, newIndexWriterConfig()
+        .setMergePolicy(newLogMergePolicy(random().nextBoolean())));
+
+    Document doc = new Document();
+    LatLonPoint point = new LatLonPoint("foo", 0., 0.);
+    doc.add(point);
+    LatLonDocValuesField docValue = new LatLonDocValuesField("foo", 0., 0.);
+    doc.add(docValue);
+
+    int numDocs = atLeast(10000);
+    for (int i = 0; i < numDocs; ++i) {
+      double lat = random().nextDouble() * 180 - 90;
+      double lon = random().nextDouble() * 360 - 180;
+      point.setLocationValue(lat, lon);
+      docValue.setLocationValue(lat, lon);
+      w.addDocument(doc);
+    }
+
+    DirectoryReader reader = w.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+
+    double lat = random().nextDouble() * 180 - 90;
+    double lon = random().nextDouble() * 360 - 180;
+    double  pivotDistance = random().nextDouble() * random().nextDouble() * GeoUtils.EARTH_MEAN_RADIUS_METERS * Math.PI;
+    float boost = (1 + random().nextInt(10)) / 3f;
+
+    Query query1 = LatLonPoint.newDistanceFeatureQuery("foo", boost, lat, lon, pivotDistance);
+    Sort sort1 = new Sort(SortField.FIELD_SCORE, LatLonDocValuesField.newDistanceSort("foo", lat, lon));
+
+    Query query2 = new MatchAllDocsQuery();
+    Sort sort2 = new Sort(LatLonDocValuesField.newDistanceSort("foo", lat, lon));
+
+    TopDocs topDocs1 = searcher.search(query1, 10, sort1);
+    TopDocs topDocs2 = searcher.search(query2, 10, sort2);
+    for (int i =0; i< 10; i++) {
+      assertTrue(topDocs1.scoreDocs[i].doc == topDocs2.scoreDocs[i].doc);
+    }
+    reader.close();
+    w.close();
+    dir.close();
+  }
+}