You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucenenet.apache.org by sy...@apache.org on 2012/09/22 19:49:00 UTC

svn commit: r1388853 - in /lucene.net/trunk/src/contrib/Spatial: ./ BBox/ Prefix/ Util/ Vector/

Author: synhershko
Date: Sat Sep 22 17:48:59 2012
New Revision: 1388853

URL: http://svn.apache.org/viewvc?rev=1388853&view=rev
Log:
LUCENE-4208 makeQuery return ConstantScoreQuery, standardize makeDistanceValueSource behavior

Added:
    lucene.net/trunk/src/contrib/Spatial/BBox/DistanceSimilarity.cs
    lucene.net/trunk/src/contrib/Spatial/Util/ReciprocalFloatFunction.cs
Modified:
    lucene.net/trunk/src/contrib/Spatial/BBox/BBoxSimilarityValueSource.cs
    lucene.net/trunk/src/contrib/Spatial/BBox/BBoxStrategy.cs
    lucene.net/trunk/src/contrib/Spatial/Contrib.Spatial.NTS.csproj
    lucene.net/trunk/src/contrib/Spatial/Contrib.Spatial.csproj
    lucene.net/trunk/src/contrib/Spatial/Prefix/PrefixTreeStrategy.cs
    lucene.net/trunk/src/contrib/Spatial/SpatialStrategy.cs
    lucene.net/trunk/src/contrib/Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
    lucene.net/trunk/src/contrib/Spatial/Vector/DistanceValueSource.cs
    lucene.net/trunk/src/contrib/Spatial/Vector/TwoDoublesStrategy.cs

Modified: lucene.net/trunk/src/contrib/Spatial/BBox/BBoxSimilarityValueSource.cs
URL: http://svn.apache.org/viewvc/lucene.net/trunk/src/contrib/Spatial/BBox/BBoxSimilarityValueSource.cs?rev=1388853&r1=1388852&r2=1388853&view=diff
==============================================================================
--- lucene.net/trunk/src/contrib/Spatial/BBox/BBoxSimilarityValueSource.cs (original)
+++ lucene.net/trunk/src/contrib/Spatial/BBox/BBoxSimilarityValueSource.cs Sat Sep 22 17:48:59 2012
@@ -60,20 +60,23 @@ namespace Lucene.Net.Spatial.BBox
 				validMaxX = FieldCache_Fields.DEFAULT.GetDocsWithField(reader, enclosingInstance.strategy.field_maxX);
 			}
 
-			public override float FloatVal(int doc)
-			{
-				// make sure it has minX and area
-				if (validMinX.Get(doc) && validMaxX.Get(doc))
-				{
-					rect.Reset(
-						minX[doc], maxX[doc],
-						minY[doc], maxY[doc]);
-					return (float)_enclosingInstance.similarity.Score(rect, null);
-				}
-				return 0;
-			}
+            public override float FloatVal(int doc)
+            {
+                // make sure it has minX and area
+                if (validMinX.Get(doc) && validMaxX.Get(doc))
+                {
+                    rect.Reset(
+                        minX[doc], maxX[doc],
+                        minY[doc], maxY[doc]);
+                    return (float) _enclosingInstance.similarity.Score(rect, null);
+                }
+                else
+                {
+                    return (float) _enclosingInstance.similarity.Score(null, null);
+                }
+            }
 
-			public override Explanation Explain(int doc)
+		    public override Explanation Explain(int doc)
 			{
 				// make sure it has minX and area
 				if (validMinX.Get(doc) && validMaxX.Get(doc))

Modified: lucene.net/trunk/src/contrib/Spatial/BBox/BBoxStrategy.cs
URL: http://svn.apache.org/viewvc/lucene.net/trunk/src/contrib/Spatial/BBox/BBoxStrategy.cs?rev=1388853&r1=1388852&r2=1388853&view=diff
==============================================================================
--- lucene.net/trunk/src/contrib/Spatial/BBox/BBoxStrategy.cs (original)
+++ lucene.net/trunk/src/contrib/Spatial/BBox/BBoxStrategy.cs Sat Sep 22 17:48:59 2012
@@ -27,50 +27,50 @@ using Spatial4n.Core.Shapes;
 
 namespace Lucene.Net.Spatial.BBox
 {
-	public class BBoxStrategy : SpatialStrategy
-	{
-		public static String SUFFIX_MINX = "__minX";
-		public static String SUFFIX_MAXX = "__maxX";
-		public static String SUFFIX_MINY = "__minY";
-		public static String SUFFIX_MAXY = "__maxY";
-		public static String SUFFIX_XDL = "__xdl";
+    public class BBoxStrategy : SpatialStrategy
+    {
+        public static String SUFFIX_MINX = "__minX";
+        public static String SUFFIX_MAXX = "__maxX";
+        public static String SUFFIX_MINY = "__minY";
+        public static String SUFFIX_MAXY = "__maxY";
+        public static String SUFFIX_XDL = "__xdl";
 
-		/*
+        /*
 		 * The Bounding Box gets stored as four fields for x/y min/max and a flag
 		 * that says if the box crosses the dateline (xdl).
 		 */
-		public readonly String field_bbox;
-		public readonly String field_minX;
-		public readonly String field_minY;
-		public readonly String field_maxX;
-		public readonly String field_maxY;
-		public readonly String field_xdl; // crosses dateline
-
-		public readonly double queryPower = 1.0;
-		public readonly double targetPower = 1.0f;
-		public int precisionStep = 8; // same as solr default
+        public readonly String field_bbox;
+        public readonly String field_minX;
+        public readonly String field_minY;
+        public readonly String field_maxX;
+        public readonly String field_maxY;
+        public readonly String field_xdl; // crosses dateline
+
+        public readonly double queryPower = 1.0;
+        public readonly double targetPower = 1.0f;
+        public int precisionStep = 8; // same as solr default
 
-		public BBoxStrategy(SpatialContext ctx, String fieldNamePrefix)
-			: base(ctx, fieldNamePrefix)
-		{
-			field_bbox = fieldNamePrefix;
-			field_minX = fieldNamePrefix + SUFFIX_MINX;
-			field_maxX = fieldNamePrefix + SUFFIX_MAXX;
-			field_minY = fieldNamePrefix + SUFFIX_MINY;
-			field_maxY = fieldNamePrefix + SUFFIX_MAXY;
-			field_xdl = fieldNamePrefix + SUFFIX_XDL;
-		}
+        public BBoxStrategy(SpatialContext ctx, String fieldNamePrefix)
+            : base(ctx, fieldNamePrefix)
+        {
+            field_bbox = fieldNamePrefix;
+            field_minX = fieldNamePrefix + SUFFIX_MINX;
+            field_maxX = fieldNamePrefix + SUFFIX_MAXX;
+            field_minY = fieldNamePrefix + SUFFIX_MINY;
+            field_maxY = fieldNamePrefix + SUFFIX_MAXY;
+            field_xdl = fieldNamePrefix + SUFFIX_XDL;
+        }
 
-		public void SetPrecisionStep(int p)
-		{
-			precisionStep = p;
-			if (precisionStep <= 0 || precisionStep >= 64)
-				precisionStep = int.MaxValue;
-		}
+        public void SetPrecisionStep(int p)
+        {
+            precisionStep = p;
+            if (precisionStep <= 0 || precisionStep >= 64)
+                precisionStep = int.MaxValue;
+        }
 
-		//---------------------------------
-		// Indexing
-		//---------------------------------
+        //---------------------------------
+        // Indexing
+        //---------------------------------
 
         public override AbstractField[] CreateIndexableFields(Shape shape)
         {
@@ -80,48 +80,58 @@ namespace Lucene.Net.Spatial.BBox
             throw new ArgumentException("Can only index Rectangle, not " + shape, "shape");
         }
 
-	    public AbstractField[] CreateIndexableFields(Rectangle bbox)
+        public AbstractField[] CreateIndexableFields(Rectangle bbox)
         {
-			var fields = new AbstractField[5];
-			fields[0] = DoubleField(field_minX, bbox.GetMinX());
-			fields[1] = DoubleField(field_maxX, bbox.GetMaxX());
-			fields[2] = DoubleField(field_minY, bbox.GetMinY());
-			fields[3] = DoubleField(field_maxY, bbox.GetMaxY());
-			fields[4] = new Field(field_xdl, bbox.GetCrossesDateLine() ? "T" : "F", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS) {OmitNorms = true, OmitTermFreqAndPositions = true};
-			return fields;
-		}
+            var fields = new AbstractField[5];
+            fields[0] = DoubleField(field_minX, bbox.GetMinX());
+            fields[1] = DoubleField(field_maxX, bbox.GetMaxX());
+            fields[2] = DoubleField(field_minY, bbox.GetMinY());
+            fields[3] = DoubleField(field_maxY, bbox.GetMaxY());
+            fields[4] = new Field(field_xdl, bbox.GetCrossesDateLine() ? "T" : "F", Field.Store.NO,
+                                  Field.Index.NOT_ANALYZED_NO_NORMS) {OmitNorms = true, OmitTermFreqAndPositions = true};
+            return fields;
+        }
 
-		private AbstractField DoubleField(string field, double value)
-		{
-			var f = new NumericField(field, precisionStep, Field.Store.NO, true) {OmitNorms = true, OmitTermFreqAndPositions = true};
-			f.SetDoubleValue(value);
-			return f;
-		}
+        private AbstractField DoubleField(string field, double value)
+        {
+            var f = new NumericField(field, precisionStep, Field.Store.NO, true)
+                        {OmitNorms = true, OmitTermFreqAndPositions = true};
+            f.SetDoubleValue(value);
+            return f;
+        }
 
-		public override ValueSource MakeValueSource(SpatialArgs args)
-		{
-            var rect = args.Shape as Rectangle;
-            if (rect == null)
-                throw new ArgumentException("Can only get valueSource by Rectangle, not " + args.Shape);
-            return new BBoxSimilarityValueSource(this, new AreaSimilarity(rect, queryPower, targetPower));
-		}
+        public override ValueSource MakeDistanceValueSource(Point queryPoint)
+        {
+            return new BBoxSimilarityValueSource(this, new DistanceSimilarity(this.GetSpatialContext(), queryPoint));
+        }
 
-		public override Query MakeQuery(SpatialArgs args)
-		{
-			var bq = new BooleanQuery();
-			var spatial = MakeFilter(args);
-			bq.Add(new ConstantScoreQuery(spatial), Occur.MUST);
+        public ValueSource MakeBBoxAreaSimilarityValueSource(Rectangle queryBox)
+        {
+            return new BBoxSimilarityValueSource(
+                this, new AreaSimilarity(queryBox, queryPower, targetPower));
+        }
 
-			// This part does the scoring
-			Query spatialRankingQuery = new FunctionQuery(MakeValueSource(args));
-			bq.Add(spatialRankingQuery, Occur.MUST);
-			return bq;
-		}
+        public override ConstantScoreQuery MakeQuery(SpatialArgs args)
+        {
+            return new ConstantScoreQuery(new QueryWrapperFilter(MakeSpatialQuery(args)));
+        }
+
+        public Query MakeQueryWithValueSource(SpatialArgs args, ValueSource valueSource)
+        {
+
+            var bq = new BooleanQuery();
+            var spatial = MakeFilter(args);
+            bq.Add(new ConstantScoreQuery(spatial), Occur.MUST);
+
+            // This part does the scoring
+            Query spatialRankingQuery = new FunctionQuery(valueSource);
+            bq.Add(spatialRankingQuery, Occur.MUST);
+            return bq;
+        }
 
-		public override Filter MakeFilter(SpatialArgs args)
+        public override Filter MakeFilter(SpatialArgs args)
 		{
-			Query spatial = MakeSpatialQuery(args);
-			return new QueryWrapperFilter(spatial);
+            return new QueryWrapperFilter(MakeSpatialQuery(args));
 		}
 
 		private Query MakeSpatialQuery(SpatialArgs args)

Added: lucene.net/trunk/src/contrib/Spatial/BBox/DistanceSimilarity.cs
URL: http://svn.apache.org/viewvc/lucene.net/trunk/src/contrib/Spatial/BBox/DistanceSimilarity.cs?rev=1388853&view=auto
==============================================================================
--- lucene.net/trunk/src/contrib/Spatial/BBox/DistanceSimilarity.cs (added)
+++ lucene.net/trunk/src/contrib/Spatial/BBox/DistanceSimilarity.cs Sat Sep 22 17:48:59 2012
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+
+using Lucene.Net.Search;
+using Spatial4n.Core.Context;
+using Spatial4n.Core.Distance;
+using Spatial4n.Core.Shapes;
+
+namespace Lucene.Net.Spatial.BBox
+{
+    /// <summary>
+    /// Returns the distance between the center of the indexed rectangle and the
+    /// query shape.
+    /// </summary>
+    public class DistanceSimilarity : BBoxSimilarity
+    {
+        private readonly Point queryPoint;
+        private readonly DistanceCalculator distCalc;
+        private readonly double nullValue;
+
+        public DistanceSimilarity(SpatialContext ctx, Point queryPoint)
+        {
+            this.queryPoint = queryPoint;
+            this.distCalc = ctx.GetDistCalc();
+            this.nullValue = (ctx.IsGeo() ? 180 : double.MaxValue);
+        }
+
+        public double Score(Rectangle indexRect, Explanation exp)
+        {
+            double score;
+            if (indexRect == null)
+            {
+                score = nullValue;
+            }
+            else
+            {
+                score = distCalc.Distance(queryPoint, indexRect.GetCenter());
+            }
+            if (exp != null)
+            {
+                exp.Value = (float) score;
+                exp.Description = GetType().Name;
+                exp.AddDetail(new Explanation(-1f, "" + queryPoint));
+                exp.AddDetail(new Explanation(-1f, "" + indexRect));
+            }
+            return score;
+        }
+    }
+}

Modified: lucene.net/trunk/src/contrib/Spatial/Contrib.Spatial.NTS.csproj
URL: http://svn.apache.org/viewvc/lucene.net/trunk/src/contrib/Spatial/Contrib.Spatial.NTS.csproj?rev=1388853&r1=1388852&r2=1388853&view=diff
==============================================================================
--- lucene.net/trunk/src/contrib/Spatial/Contrib.Spatial.NTS.csproj (original)
+++ lucene.net/trunk/src/contrib/Spatial/Contrib.Spatial.NTS.csproj Sat Sep 22 17:48:59 2012
@@ -127,6 +127,7 @@
     <Compile Include="BBox\BBoxSimilarity.cs" />
     <Compile Include="BBox\BBoxSimilarityValueSource.cs" />
     <Compile Include="BBox\BBoxStrategy.cs" />
+    <Compile Include="BBox\DistanceSimilarity.cs" />
     <Compile Include="Prefix\PointPrefixTreeFieldCacheProvider.cs" />
     <Compile Include="Prefix\PrefixTreeStrategy.cs" />
     <Compile Include="Prefix\RecursivePrefixTreeFilter.cs" />
@@ -144,6 +145,7 @@
     <Compile Include="Queries\UnsupportedSpatialOperation.cs" />
     <Compile Include="SpatialStrategy.cs" />
     <Compile Include="Util\Bits.cs" />
+    <Compile Include="Util\ReciprocalFloatFunction.cs" />
     <Compile Include="Util\ShapeFieldCacheDistanceValueSource.cs" />
     <Compile Include="Util\CachingDoubleValueSource.cs" />
     <Compile Include="Util\CompatibilityExtensions.cs" />

Modified: lucene.net/trunk/src/contrib/Spatial/Contrib.Spatial.csproj
URL: http://svn.apache.org/viewvc/lucene.net/trunk/src/contrib/Spatial/Contrib.Spatial.csproj?rev=1388853&r1=1388852&r2=1388853&view=diff
==============================================================================
--- lucene.net/trunk/src/contrib/Spatial/Contrib.Spatial.csproj (original)
+++ lucene.net/trunk/src/contrib/Spatial/Contrib.Spatial.csproj Sat Sep 22 17:48:59 2012
@@ -127,6 +127,7 @@
     <Compile Include="BBox\BBoxSimilarity.cs" />
     <Compile Include="BBox\BBoxSimilarityValueSource.cs" />
     <Compile Include="BBox\BBoxStrategy.cs" />
+    <Compile Include="BBox\DistanceSimilarity.cs" />
     <Compile Include="Prefix\PointPrefixTreeFieldCacheProvider.cs" />
     <Compile Include="Prefix\PrefixTreeStrategy.cs" />
     <Compile Include="Prefix\RecursivePrefixTreeFilter.cs" />
@@ -144,6 +145,7 @@
     <Compile Include="Queries\UnsupportedSpatialOperation.cs" />
     <Compile Include="SpatialStrategy.cs" />
     <Compile Include="Util\Bits.cs" />
+    <Compile Include="Util\ReciprocalFloatFunction.cs" />
     <Compile Include="Util\ShapeFieldCacheDistanceValueSource.cs" />
     <Compile Include="Util\CachingDoubleValueSource.cs" />
     <Compile Include="Util\CompatibilityExtensions.cs" />

Modified: lucene.net/trunk/src/contrib/Spatial/Prefix/PrefixTreeStrategy.cs
URL: http://svn.apache.org/viewvc/lucene.net/trunk/src/contrib/Spatial/Prefix/PrefixTreeStrategy.cs?rev=1388853&r1=1388852&r2=1388853&view=diff
==============================================================================
--- lucene.net/trunk/src/contrib/Spatial/Prefix/PrefixTreeStrategy.cs (original)
+++ lucene.net/trunk/src/contrib/Spatial/Prefix/PrefixTreeStrategy.cs Sat Sep 22 17:48:59 2012
@@ -142,12 +142,6 @@ namespace Lucene.Net.Spatial.Prefix
 			}
 		}
 
-		public override ValueSource MakeValueSource(SpatialArgs args)
-		{
-			var calc = grid.GetSpatialContext().GetDistCalc();
-			return MakeValueSource(args, calc);
-		}
-
 		public ShapeFieldCacheProvider<Point> GetCacheProvider()
 		{
 			PointPrefixTreeFieldCacheProvider p;
@@ -165,11 +159,10 @@ namespace Lucene.Net.Spatial.Prefix
 			return p;
 		}
 
-		public ValueSource MakeValueSource(SpatialArgs args, DistanceCalculator calc)
+        public override ValueSource MakeDistanceValueSource(Point queryPoint)
 		{
 			var p = (PointPrefixTreeFieldCacheProvider)GetCacheProvider();
-			Point point = args.Shape.GetCenter();
-			return new ShapeFieldCacheDistanceValueSource(point, calc, p);
+            return new ShapeFieldCacheDistanceValueSource(ctx, p, queryPoint);
 		}
 
 		public SpatialPrefixTree GetGrid()

Modified: lucene.net/trunk/src/contrib/Spatial/SpatialStrategy.cs
URL: http://svn.apache.org/viewvc/lucene.net/trunk/src/contrib/Spatial/SpatialStrategy.cs?rev=1388853&r1=1388852&r2=1388853&view=diff
==============================================================================
--- lucene.net/trunk/src/contrib/Spatial/SpatialStrategy.cs (original)
+++ lucene.net/trunk/src/contrib/Spatial/SpatialStrategy.cs Sat Sep 22 17:48:59 2012
@@ -89,37 +89,59 @@ namespace Lucene.Net.Spatial
 		}
 
 		/// <summary>
-		/// The value source yields a number that is proportional to the distance between the query shape and indexed data.
+		/// 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.
 		/// </summary>
-		/// <param name="args"></param>
-		/// <param name="fieldInfo"></param>
-		/// <returns></returns>
-		public abstract ValueSource MakeValueSource(SpatialArgs args);
+		public abstract ValueSource MakeDistanceValueSource(Point queryPoint);
 
-		/// <summary>
-		/// Make a query which has a score based on the distance from the data to the query shape.
-		/// The default implementation constructs a {@link FilteredQuery} based on
-		/// {@link #makeFilter(com.spatial4j.core.query.SpatialArgs, SpatialFieldInfo)} and
-		/// {@link #makeValueSource(com.spatial4j.core.query.SpatialArgs, SpatialFieldInfo)}.
-		/// </summary>
-		/// <param name="args"></param>
-		/// <param name="fieldInfo"></param>
-		/// <returns></returns>
-		public virtual Query MakeQuery(SpatialArgs args)
+	    /// <summary>
+	    /// Make a (ConstantScore) Query based principally on {@link org.apache.lucene.spatial.query.SpatialOperation}
+	    /// and {@link Shape} from the supplied {@code args}.
+	    /// The default implementation is
+	    /// <pre>return new ConstantScoreQuery(makeFilter(args));</pre>
+	    /// </summary>
+	    /// <param name="args"></param>
+	    /// <returns></returns>
+	    public virtual ConstantScoreQuery MakeQuery(SpatialArgs args)
 		{
-			Filter filter = MakeFilter(args);
-			ValueSource vs = MakeValueSource(args);
-			return new FilteredQuery(new FunctionQuery(vs), filter);
+            return new ConstantScoreQuery(MakeFilter(args));
 		}
 
 		/// <summary>
-		/// Make a Filter
+		/// Make a Filter based principally on {@link org.apache.lucene.spatial.query.SpatialOperation}
+		/// and {@link Shape} from the supplied {@code args}.
+		/// <p />
+		/// If a subclasses implements
+		/// {@link #makeQuery(org.apache.lucene.spatial.query.SpatialArgs)}
+		/// then this method could be simply:
+		/// <pre>return new QueryWrapperFilter(makeQuery(args).getQuery());</pre>
 		/// </summary>
 		/// <param name="args"></param>
 		/// <returns></returns>
 		public abstract Filter MakeFilter(SpatialArgs args);
 
-		public override string ToString()
+        /// <summary>
+        /// Returns a ValueSource with values ranging from 1 to 0, depending inversely
+        /// on the distance from {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point)}.
+        /// The formula is <code>c/(d + c)</code> 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
+        /// low as ~0.1 at its furthest edges.
+        /// </summary>
+        /// <param name="queryShape"></param>
+        /// <returns></returns>
+        public ValueSource MakeRecipDistanceValueSource(Shape queryShape)
+        {
+            Rectangle bbox = queryShape.GetBoundingBox();
+            double diagonalDist = ctx.GetDistCalc().Distance(
+                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);
+        }
+
+	    public override string ToString()
 		{
 			return GetType().Name + " field:" + fieldName + " ctx=" + ctx;
 		}

Added: lucene.net/trunk/src/contrib/Spatial/Util/ReciprocalFloatFunction.cs
URL: http://svn.apache.org/viewvc/lucene.net/trunk/src/contrib/Spatial/Util/ReciprocalFloatFunction.cs?rev=1388853&view=auto
==============================================================================
--- lucene.net/trunk/src/contrib/Spatial/Util/ReciprocalFloatFunction.cs (added)
+++ lucene.net/trunk/src/contrib/Spatial/Util/ReciprocalFloatFunction.cs Sat Sep 22 17:48:59 2012
@@ -0,0 +1,102 @@
+/*
+ * 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.
+ */
+
+using System;
+using Lucene.Net.Index;
+using Lucene.Net.Search.Function;
+
+namespace Lucene.Net.Spatial.Util
+{
+    public class ReciprocalFloatFunction : ValueSource
+    {
+        protected readonly ValueSource source;
+        protected readonly float m;
+        protected readonly float a;
+        protected readonly float b;
+
+        /// <summary>
+        /// f(source) = a/(m*float(source)+b)
+        /// </summary>
+        /// <param name="source"></param>
+        /// <param name="m"></param>
+        /// <param name="a"></param>
+        /// <param name="b"></param>
+        public ReciprocalFloatFunction(ValueSource source, float m, float a, float b)
+        {
+            this.source = source;
+            this.m = m;
+            this.a = a;
+            this.b = b;
+        }
+
+        public class FloatDocValues : DocValues
+        {
+            private readonly ReciprocalFloatFunction _enclosingInstance;
+            private readonly DocValues vals;
+
+            public FloatDocValues(ReciprocalFloatFunction enclosingInstance, DocValues vals)
+            {
+                _enclosingInstance = enclosingInstance;
+                this.vals = vals;
+            }
+
+            public override float FloatVal(int doc)
+            {
+                return _enclosingInstance.a / (_enclosingInstance.m * vals.FloatVal(doc) + _enclosingInstance.b);
+            }
+
+            public override string ToString(int doc)
+            {
+                return _enclosingInstance.a + "/("
+                       + _enclosingInstance.m + "*float(" + vals.ToString(doc) + ')'
+                       + '+' + _enclosingInstance.b + ')';
+            }
+        }
+
+        public override DocValues GetValues(IndexReader reader)
+        {
+            var vals = source.GetValues(reader);
+            return new FloatDocValues(this, vals);
+        }
+
+        public override string Description()
+        {
+            return a + "/("
+                   + m + "*float(" + source.Description() + ")"
+                   + "+" + b + ')';
+        }
+
+        public override bool Equals(object o)
+        {
+            if (typeof(ReciprocalFloatFunction) != o.GetType()) return false;
+            var other = (ReciprocalFloatFunction)o;
+            return this.m == other.m
+                   && this.a == other.a
+                   && this.b == other.b
+                   && this.source.Equals(other.source);
+        }
+
+        public override int GetHashCode()
+        {
+            throw new NotImplementedException();
+            //TODO
+            //int h = Float.floatToIntBits(a) + Float.floatToIntBits(m);
+            //h ^= (h << 13) | (h >>> 20);
+            //return h + (Float.floatToIntBits(b)) + source.hashCode();
+        }
+    }
+}

Modified: lucene.net/trunk/src/contrib/Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
URL: http://svn.apache.org/viewvc/lucene.net/trunk/src/contrib/Spatial/Util/ShapeFieldCacheDistanceValueSource.cs?rev=1388853&r1=1388852&r2=1388853&view=diff
==============================================================================
--- lucene.net/trunk/src/contrib/Spatial/Util/ShapeFieldCacheDistanceValueSource.cs (original)
+++ lucene.net/trunk/src/contrib/Spatial/Util/ShapeFieldCacheDistanceValueSource.cs Sat Sep 22 17:48:59 2012
@@ -18,6 +18,7 @@
 using System;
 using Lucene.Net.Index;
 using Lucene.Net.Search.Function;
+using Spatial4n.Core.Context;
 using Spatial4n.Core.Distance;
 using Spatial4n.Core.Shapes;
 
@@ -29,25 +30,32 @@ namespace Lucene.Net.Spatial.Util
 	public class ShapeFieldCacheDistanceValueSource : ValueSource
 	{
 		private readonly ShapeFieldCacheProvider<Point> provider;
-		private readonly DistanceCalculator calculator;
+		private readonly SpatialContext ctx;
 		private readonly Point from;
 
-		public ShapeFieldCacheDistanceValueSource(Point from, DistanceCalculator calc, ShapeFieldCacheProvider<Point> provider)
+		public ShapeFieldCacheDistanceValueSource(SpatialContext ctx, ShapeFieldCacheProvider<Point> provider, Point from)
 		{
+            this.ctx = ctx;
 			this.from = from;
 			this.provider = provider;
-			this.calculator = calc;
 		}
 
 		public class CachedDistanceDocValues : DocValues
 		{
 			private readonly ShapeFieldCacheDistanceValueSource enclosingInstance;
 			private readonly ShapeFieldCache<Point> cache;
+		    private readonly Point from;
+		    private readonly DistanceCalculator calculator;
+		    private readonly double nullValue;
 
-			public CachedDistanceDocValues(ShapeFieldCache<Point> cache, ShapeFieldCacheDistanceValueSource enclosingInstance)
+			public CachedDistanceDocValues(IndexReader reader, ShapeFieldCacheDistanceValueSource enclosingInstance)
 			{
+                cache = enclosingInstance.provider.GetCache(reader);
 				this.enclosingInstance = enclosingInstance;
-				this.cache = cache;
+				
+                from = enclosingInstance.from;
+			    calculator = enclosingInstance.ctx.GetDistCalc();
+			    nullValue = (enclosingInstance.ctx.IsGeo() ? 180 : double.MaxValue);
 			}
 
 			public override float FloatVal(int doc)
@@ -55,22 +63,22 @@ namespace Lucene.Net.Spatial.Util
 				return (float)DoubleVal(doc);
 			}
 
-			public override double DoubleVal(int doc)
-			{
-				var vals = cache.GetShapes(doc);
-				if (vals != null)
-				{
-					double v = enclosingInstance.calculator.Distance(enclosingInstance.from, vals[0]);
-					for (int i = 1; i < vals.Count; i++)
-					{
-						v = Math.Min(v, enclosingInstance.calculator.Distance(enclosingInstance.from, vals[i]));
-					}
-					return v;
-				}
-				return Double.NaN; // ?? maybe max?
-			}
+            public override double DoubleVal(int doc)
+            {
+                var vals = cache.GetShapes(doc);
+                if (vals != null)
+                {
+                    double v = calculator.Distance(from, vals[0]);
+                    for (int i = 1; i < vals.Count; i++)
+                    {
+                        v = Math.Min(v, calculator.Distance(from, vals[i]));
+                    }
+                    return v;
+                }
+                return nullValue;
+            }
 
-			public override string ToString(int doc)
+		    public override string ToString(int doc)
 			{
 				return enclosingInstance.Description() + "=" + FloatVal(doc);
 			}
@@ -78,13 +86,12 @@ namespace Lucene.Net.Spatial.Util
 
 		public override DocValues GetValues(IndexReader reader)
 		{
-			ShapeFieldCache<Point> cache = provider.GetCache(reader);
-			return new CachedDistanceDocValues(cache, this);
+			return new CachedDistanceDocValues(reader, this);
 		}
 
 		public override string Description()
 		{
-			return "DistanceValueSource(" + calculator + ")";
+            return GetType().Name + "(" + provider + ", " + from + ")";
 		}
 
 		public override bool Equals(object o)
@@ -94,17 +101,16 @@ namespace Lucene.Net.Spatial.Util
 			var that = o as ShapeFieldCacheDistanceValueSource;
 
 			if (that == null) return false;
-			if (calculator != null ? !calculator.Equals(that.calculator) : that.calculator != null) return false;
-			if (from != null ? !from.Equals(that.from) : that.from != null) return false;
+            if (!ctx.Equals(that.ctx)) return false;
+            if (!from.Equals(that.from)) return false;
+            if (!provider.Equals(that.provider)) return false;
 
 			return true;
 		}
 
 		public override int GetHashCode()
 		{
-			var result = calculator != null ? calculator.GetHashCode() : 0;
-			result = 31 * result + (from != null ? from.GetHashCode() : 0);
-			return result;
+		    return from.GetHashCode();
 		}
 	}
 }

Modified: lucene.net/trunk/src/contrib/Spatial/Vector/DistanceValueSource.cs
URL: http://svn.apache.org/viewvc/lucene.net/trunk/src/contrib/Spatial/Vector/DistanceValueSource.cs?rev=1388853&r1=1388852&r2=1388853&view=diff
==============================================================================
--- lucene.net/trunk/src/contrib/Spatial/Vector/DistanceValueSource.cs (original)
+++ lucene.net/trunk/src/contrib/Spatial/Vector/DistanceValueSource.cs Sat Sep 22 17:48:59 2012
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+using System.Diagnostics;
 using Lucene.Net.Index;
 using Lucene.Net.Search;
 using Lucene.Net.Search.Function;
@@ -26,19 +27,17 @@ using Spatial4n.Core.Shapes.Impl;
 namespace Lucene.Net.Spatial.Vector
 {
 	/// <summary>
-	/// An implementation of the Lucene ValueSource model to support spatial relevance ranking.
+    /// An implementation of the Lucene ValueSource model that returns the distance.
 	/// </summary>
 	public class DistanceValueSource : ValueSource
 	{
-		private TwoDoublesStrategy strategy;
+		private readonly TwoDoublesStrategy strategy;
 		private readonly Point from;
-		private readonly DistanceCalculator calculator;
 
-		public DistanceValueSource(TwoDoublesStrategy strategy, Point from, DistanceCalculator calc)
+		public DistanceValueSource(TwoDoublesStrategy strategy, Point from)
 		{
 			this.strategy = strategy;
 			this.from = from;
-			this.calculator = calc;
 		}
 
 		public class DistanceDocValues : DocValues
@@ -48,6 +47,10 @@ namespace Lucene.Net.Spatial.Vector
 			private readonly double[] ptX, ptY;
 			private readonly IBits validX, validY;
 
+            private readonly Point from;
+            private readonly DistanceCalculator calculator;
+            private readonly double nullValue;
+
 			public DistanceDocValues(DistanceValueSource enclosingInstance, IndexReader reader)
 			{
 				this.enclosingInstance = enclosingInstance;
@@ -56,6 +59,10 @@ namespace Lucene.Net.Spatial.Vector
 				ptY = FieldCache_Fields.DEFAULT.GetDoubles(reader, enclosingInstance.strategy.GetFieldNameY()/*, true*/);
 				validX = FieldCache_Fields.DEFAULT.GetDocsWithField(reader, enclosingInstance.strategy.GetFieldNameX());
 				validY = FieldCache_Fields.DEFAULT.GetDocsWithField(reader, enclosingInstance.strategy.GetFieldNameY());
+
+                from = enclosingInstance.from;
+                calculator = enclosingInstance.strategy.GetSpatialContext().GetDistCalc();
+                nullValue = (enclosingInstance.strategy.GetSpatialContext().IsGeo() ? 180 : double.MaxValue);
 			}
 
 			public override float FloatVal(int doc)
@@ -66,11 +73,12 @@ namespace Lucene.Net.Spatial.Vector
 			public override double DoubleVal(int doc)
 			{
 				// make sure it has minX and area
-				if (validX.Get(doc) && validY.Get(doc))
+				if (validX.Get(doc))
 				{
-					return enclosingInstance.calculator.Distance(enclosingInstance.from, ptX[doc], ptY[doc]);
+				    Debug.Assert(validY.Get(doc));
+					return calculator.Distance(enclosingInstance.from, ptX[doc], ptY[doc]);
 				}
-				return 0;
+				return nullValue;
 			}
 
 			public override string ToString(int doc)
@@ -86,7 +94,7 @@ namespace Lucene.Net.Spatial.Vector
 
 		public override string Description()
 		{
-			return "DistanceValueSource(" + calculator + ")";
+            return "DistanceValueSource(" + strategy + ", " + from + ")";
 		}
 
 		public override bool Equals(object o)
@@ -96,19 +104,15 @@ namespace Lucene.Net.Spatial.Vector
 			var that = o as DistanceValueSource;
 			if (that == null) return false;
 
-			if (calculator != null ? !calculator.Equals(that.calculator) : that.calculator != null) return false;
-			if (strategy != null ? !strategy.Equals(that.strategy) : that.strategy != null) return false;
-			if (from != null ? !from.Equals(that.from) : that.from != null) return false;
+            if (!from.Equals(that.from)) return false;
+            if (!strategy.Equals(that.strategy)) return false;
 
 			return true;
 		}
 
 		public override int GetHashCode()
 		{
-			int result = strategy != null ? strategy.GetHashCode() : 0;
-			result = 31 * result + (calculator != null ? calculator.GetHashCode() : 0);
-			result = 31 * result + (from != null ? from.GetHashCode() : 0);
-			return result;
+		    return from.GetHashCode();
 		}
 	}
 }

Modified: lucene.net/trunk/src/contrib/Spatial/Vector/TwoDoublesStrategy.cs
URL: http://svn.apache.org/viewvc/lucene.net/trunk/src/contrib/Spatial/Vector/TwoDoublesStrategy.cs?rev=1388853&r1=1388852&r2=1388853&view=diff
==============================================================================
--- lucene.net/trunk/src/contrib/Spatial/Vector/TwoDoublesStrategy.cs (original)
+++ lucene.net/trunk/src/contrib/Spatial/Vector/TwoDoublesStrategy.cs Sat Sep 22 17:48:59 2012
@@ -87,15 +87,31 @@ namespace Lucene.Net.Spatial.Vector
 				return f;
 		}
 
-		public override ValueSource MakeValueSource(SpatialArgs args)
+		public override ValueSource MakeDistanceValueSource(Point queryPoint)
 		{
-			Point p = args.Shape.GetCenter();
-			return new DistanceValueSource(this, p, ctx.GetDistCalc());
+            return new DistanceValueSource(this, queryPoint);
 		}
 
-		public override Query MakeQuery(SpatialArgs args)
+		public override ConstantScoreQuery MakeQuery(SpatialArgs args)
 		{
-			// For starters, just limit the bbox
+		    if (! SpatialOperation.Is(args.Operation,
+		                              SpatialOperation.Intersects,
+		                              SpatialOperation.IsWithin))
+		        throw new UnsupportedSpatialOperation(args.Operation);
+		    Shape shape = args.Shape;
+		    var rect = shape as Rectangle;
+		    if (rect == null)
+		        throw new InvalidShapeException("Only Rectangle is currently supported, got " + shape.GetType());
+		    var bbox = (Rectangle) shape;
+		    if (bbox.GetCrossesDateLine())
+		        throw new InvalidOperationException("Crossing dateline not yet supported");
+
+		    return new ConstantScoreQuery(new QueryWrapperFilter(MakeWithin(bbox)));
+		}
+
+        public Query MakeQueryDistanceScore(SpatialArgs args)
+        {
+	        // For starters, just limit the bbox
 			var shape = args.Shape;
 			if (!(shape is Rectangle || shape is Circle))
 				throw new InvalidShapeException("Only Rectangles and Circles are currently supported, found ["
@@ -127,7 +143,7 @@ namespace Lucene.Net.Spatial.Vector
 				if (circle != null)
 				{
 					// Make the ValueSource
-					valueSource = MakeValueSource(args);
+                    valueSource = MakeDistanceValueSource(shape.GetCenter());
 
 					var vsf = new ValueSourceFilter(
 						new QueryWrapperFilter(spatial), valueSource, 0, circle.GetRadius());
@@ -151,7 +167,7 @@ namespace Lucene.Net.Spatial.Vector
 			}
 			else
 			{
-				valueSource = MakeValueSource(args);
+                valueSource = MakeDistanceValueSource(shape.GetCenter());
 			}
 			Query spatialRankingQuery = new FunctionQuery(valueSource);
 			var bq = new BooleanQuery();
@@ -163,23 +179,7 @@ namespace Lucene.Net.Spatial.Vector
 
 		public override Filter MakeFilter(SpatialArgs args)
 		{
-			var circle = args.Shape as Circle;
-			if (circle != null)
-			{
-				if (SpatialOperation.Is(args.Operation,
-					SpatialOperation.Intersects,
-					SpatialOperation.IsWithin))
-				{
-					Query bbox = MakeWithin(circle.GetBoundingBox());
-
-					// Make the ValueSource
-					ValueSource valueSource = MakeValueSource(args);
-
-					return new ValueSourceFilter(
-						new QueryWrapperFilter(bbox), valueSource, 0, circle.GetRadius());
-				}
-			}
-			return new QueryWrapperFilter(MakeQuery(args));
+            return new QueryWrapperFilter(MakeQuery(args));
 		}
 
 		/// <summary>