You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ds...@apache.org on 2013/07/24 19:25:12 UTC

svn commit: r1506635 - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/spatial/ lucene/spatial/src/java/org/apache/lucene/spatial/ lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ lucene/spatial/src/java/org/apache/lucene/spatial/util/ lu...

Author: dsmiley
Date: Wed Jul 24 17:25:11 2013
New Revision: 1506635

URL: http://svn.apache.org/r1506635
Log:
LUCENE-5118: multiplier to spatial makeDistanceValueSource

Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/lucene/   (props changed)
    lucene/dev/branches/branch_4x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_4x/lucene/spatial/   (props changed)
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/SpatialStrategy.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialExample.java
    lucene/dev/branches/branch_4x/solr/   (props changed)
    lucene/dev/branches/branch_4x/solr/core/   (props changed)
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java

Modified: lucene/dev/branches/branch_4x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/CHANGES.txt?rev=1506635&r1=1506634&r2=1506635&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/lucene/CHANGES.txt Wed Jul 24 17:25:11 2013
@@ -20,6 +20,9 @@ New features
   (default is false).  If true then edits are measured in Unicode code
   points instead of UTF8 bytes.  (Artem Lukanin via Mike McCandless)
 
+* LUCENE-5118: SpatialStrategy.makeDistanceValueSource() now has an optional
+  multiplier for scaling degrees to another unit. (David Smiley)
+
 Bug Fixes
 
 * LUCENE-5116: IndexWriter.addIndexes(IndexReader...) should drop empty (or all

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/SpatialStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/SpatialStrategy.java?rev=1506635&r1=1506634&r2=1506635&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/SpatialStrategy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/SpatialStrategy.java Wed Jul 24 17:25:11 2013
@@ -102,11 +102,20 @@ public abstract class SpatialStrategy {
   public abstract Field[] createIndexableFields(Shape shape);
 
   /**
+   * See {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point, double)} called with
+   * a multiplier of 1.0 (i.e. units of degrees).
+   */
+  public ValueSource makeDistanceValueSource(Point queryPoint) {
+    return makeDistanceValueSource(queryPoint, 1.0);
+  }
+
+  /**
    * Make a ValueSource returning the distance between the center of the
    * indexed shape and {@code queryPoint}.  If there are multiple indexed shapes
-   * then the closest one is chosen.
+   * then the closest one is chosen. The result is multiplied by {@code multiplier}, which
+   * conveniently is used to get the desired units.
    */
-  public abstract ValueSource makeDistanceValueSource(Point queryPoint);
+  public abstract ValueSource makeDistanceValueSource(Point queryPoint, double multiplier);
 
   /**
    * Make a Query based principally on {@link org.apache.lucene.spatial.query.SpatialOperation}
@@ -139,7 +148,7 @@ public abstract class SpatialStrategy {
 
   /**
    * Returns a ValueSource with values ranging from 1 to 0, depending inversely
-   * on the distance from {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point)}.
+   * on the distance from {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point,double)}.
    * The formula is {@code c/(d + c)} where 'd' is the distance and 'c' is
    * one tenth the distance to the farthest edge from the center. Thus the
    * scores will be 1 for indexed points at the center of the query shape and as
@@ -151,7 +160,7 @@ public abstract class SpatialStrategy {
         ctx.makePoint(bbox.getMinX(), bbox.getMinY()), bbox.getMaxX(), bbox.getMaxY());
     double distToEdge = diagonalDist * 0.5;
     float c = (float)distToEdge * 0.1f;//one tenth
-    return new ReciprocalFloatFunction(makeDistanceValueSource(queryShape.getCenter()), 1f, c, c);
+    return new ReciprocalFloatFunction(makeDistanceValueSource(queryShape.getCenter(), 1.0), 1f, c, c);
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java?rev=1506635&r1=1506634&r2=1506635&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java Wed Jul 24 17:25:11 2013
@@ -57,7 +57,7 @@ import java.util.concurrent.ConcurrentHa
  * <li>Only {@link org.apache.lucene.spatial.query.SpatialOperation#Intersects}
  * is supported.  If only points are indexed then this is effectively equivalent
  * to IsWithin.</li>
- * <li>The strategy supports {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point)}
+ * <li>The strategy supports {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point,double)}
  * even for multi-valued data, so long as the indexed data is all points; the
  * behavior is undefined otherwise.  However, <em>it will likely be removed in
  * the future</em> in lieu of using another strategy with a more scalable
@@ -183,7 +183,7 @@ public abstract class PrefixTreeStrategy
   }
 
   @Override
-  public ValueSource makeDistanceValueSource(Point queryPoint) {
+  public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
     PointPrefixTreeFieldCacheProvider p = provider.get( getFieldName() );
     if( p == null ) {
       synchronized (this) {//double checked locking idiom is okay since provider is threadsafe
@@ -195,7 +195,7 @@ public abstract class PrefixTreeStrategy
       }
     }
 
-    return new ShapeFieldCacheDistanceValueSource(ctx, p, queryPoint);
+    return new ShapeFieldCacheDistanceValueSource(ctx, p, queryPoint, multiplier);
   }
 
   public SpatialPrefixTree getGrid() {

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java?rev=1506635&r1=1506634&r2=1506635&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java Wed Jul 24 17:25:11 2013
@@ -38,14 +38,17 @@ import java.util.Map;
  */
 public class ShapeFieldCacheDistanceValueSource extends ValueSource {
 
-  private final ShapeFieldCacheProvider<Point> provider;
   private final SpatialContext ctx;
   private final Point from;
+  private final ShapeFieldCacheProvider<Point> provider;
+  private final double multiplier;
 
-  public ShapeFieldCacheDistanceValueSource(SpatialContext ctx, ShapeFieldCacheProvider<Point> provider, Point from) {
+  public ShapeFieldCacheDistanceValueSource(SpatialContext ctx,
+      ShapeFieldCacheProvider<Point> provider, Point from, double multiplier) {
     this.ctx = ctx;
     this.from = from;
     this.provider = provider;
+    this.multiplier = multiplier;
   }
 
   @Override
@@ -60,7 +63,7 @@ public class ShapeFieldCacheDistanceValu
           provider.getCache(readerContext.reader());
       private final Point from = ShapeFieldCacheDistanceValueSource.this.from;
       private final DistanceCalculator calculator = ctx.getDistCalc();
-      private final double nullValue = (ctx.isGeo() ? 180 : Double.MAX_VALUE);
+      private final double nullValue = (ctx.isGeo() ? 180 * multiplier : Double.MAX_VALUE);
 
       @Override
       public float floatVal(int doc) {
@@ -69,13 +72,14 @@ public class ShapeFieldCacheDistanceValu
 
       @Override
       public double doubleVal(int doc) {
+
         List<Point> vals = cache.getShapes( doc );
         if( vals != null ) {
           double v = calculator.distance(from, vals.get(0));
           for( int i=1; i<vals.size(); i++ ) {
             v = Math.min(v, calculator.distance(from, vals.get(i)));
           }
-          return v;
+          return v * multiplier;
         }
         return nullValue;
       }
@@ -97,6 +101,7 @@ public class ShapeFieldCacheDistanceValu
     if (!ctx.equals(that.ctx)) return false;
     if (!from.equals(that.from)) return false;
     if (!provider.equals(that.provider)) return false;
+    if (multiplier != that.multiplier) return false;
 
     return true;
   }

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java?rev=1506635&r1=1506634&r2=1506635&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java Wed Jul 24 17:25:11 2013
@@ -39,13 +39,15 @@ public class DistanceValueSource extends
 
   private PointVectorStrategy strategy;
   private final Point from;
+  private final double multiplier;
 
   /**
    * Constructor.
    */
-  public DistanceValueSource(PointVectorStrategy strategy, Point from) {
+  public DistanceValueSource(PointVectorStrategy strategy, Point from, double multiplier) {
     this.strategy = strategy;
     this.from = from;
+    this.multiplier = multiplier;
   }
 
   /**
@@ -72,7 +74,8 @@ public class DistanceValueSource extends
 
       private final Point from = DistanceValueSource.this.from;
       private final DistanceCalculator calculator = strategy.getSpatialContext().getDistCalc();
-      private final double nullValue = (strategy.getSpatialContext().isGeo() ? 180 : Double.MAX_VALUE);
+      private final double nullValue =
+          (strategy.getSpatialContext().isGeo() ? 180 * multiplier : Double.MAX_VALUE);
 
       @Override
       public float floatVal(int doc) {
@@ -84,7 +87,7 @@ public class DistanceValueSource extends
         // make sure it has minX and area
         if (validX.get(doc)) {
           assert validY.get(doc);
-          return calculator.distance(from, ptX.get(doc), ptY.get(doc));
+          return calculator.distance(from, ptX.get(doc), ptY.get(doc)) * multiplier;
         }
         return nullValue;
       }
@@ -105,6 +108,7 @@ public class DistanceValueSource extends
 
     if (!from.equals(that.from)) return false;
     if (!strategy.equals(that.strategy)) return false;
+    if (multiplier != that.multiplier) return false;
 
     return true;
   }

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java?rev=1506635&r1=1506634&r2=1506635&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java Wed Jul 24 17:25:11 2013
@@ -120,8 +120,8 @@ public class PointVectorStrategy extends
   }
 
   @Override
-  public ValueSource makeDistanceValueSource(Point queryPoint) {
-    return new DistanceValueSource(this, queryPoint);
+  public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
+    return new DistanceValueSource(this, queryPoint, multiplier);
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java?rev=1506635&r1=1506634&r2=1506635&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java Wed Jul 24 17:25:11 2013
@@ -32,6 +32,7 @@ import org.junit.Test;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
 public class DistanceStrategyTest extends StrategyTestCase {
@@ -116,6 +117,11 @@ public class DistanceStrategyTest extend
 
   void checkDistValueSource(String ptStr, float... distances) throws IOException {
     Point pt = (Point) ctx.readShape(ptStr);
-    checkValueSource(strategy.makeDistanceValueSource(pt), distances, 1.0e-4f);
+    float multiplier = random().nextFloat() * 100f;
+    float[] dists2 = Arrays.copyOf(distances, distances.length);
+    for (int i = 0; i < dists2.length; i++) {
+      dists2[i] *= multiplier;
+    }
+    checkValueSource(strategy.makeDistanceValueSource(pt, multiplier), dists2, 1.0e-3f);
   }
 }

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialExample.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialExample.java?rev=1506635&r1=1506634&r2=1506635&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialExample.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialExample.java Wed Jul 24 17:25:11 2013
@@ -162,7 +162,8 @@ public class SpatialExample extends Luce
     //--Match all, order by distance ascending
     {
       Point pt = ctx.makePoint(60, -50);
-      ValueSource valueSource = strategy.makeDistanceValueSource(pt);//the distance (in degrees)
+      double degToKm = DistanceUtils.degrees2Dist(1, DistanceUtils.EARTH_MEAN_RADIUS_KM);
+      ValueSource valueSource = strategy.makeDistanceValueSource(pt, degToKm);//the distance (in km)
       Sort distSort = new Sort(valueSource.getSortField(false)).rewrite(indexSearcher);//false=asc dist
       TopDocs docs = indexSearcher.search(new MatchAllDocsQuery(), 10, distSort);
       assertDocMatchedIds(indexSearcher, docs, 4, 20, 2);

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java?rev=1506635&r1=1506634&r2=1506635&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java Wed Jul 24 17:25:11 2013
@@ -241,12 +241,14 @@ public abstract class AbstractSpatialFie
 
     //We get the valueSource for the score then the filter and combine them.
     ValueSource valueSource;
-    if ("distance".equals(score))
-      valueSource = strategy.makeDistanceValueSource(spatialArgs.getShape().getCenter());
-    else if ("recipDistance".equals(score))
+    if ("distance".equals(score)) {
+      double multiplier = 1.0;//TODO support units=kilometers
+      valueSource = strategy.makeDistanceValueSource(spatialArgs.getShape().getCenter(), multiplier);
+    } else if ("recipDistance".equals(score)) {
       valueSource = strategy.makeRecipDistanceValueSource(spatialArgs.getShape());
-    else
+    } else {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "'score' local-param must be one of 'none', 'distance', or 'recipDistance'");
+    }
     FunctionQuery functionQuery = new FunctionQuery(valueSource);
 
     if (localParams != null && !localParams.getBool(FILTER_PARAM, true))