You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucenenet.apache.org by ni...@apache.org on 2016/11/25 11:07:06 UTC

[01/26] lucenenet git commit: Ported over Lucene.Net.Spatial

Repository: lucenenet
Updated Branches:
  refs/heads/master 96d38effd -> e8735ed96


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs b/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
index acf6f96..73c57f9 100644
--- a/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
+++ b/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
@@ -28,32 +28,27 @@ namespace Lucene.Net.Spatial.Prefix
     /// <see cref="Lucene.Net.Spatial.Util.ShapeFieldCacheProvider{T}">Lucene.Net.Spatial.Util.ShapeFieldCacheProvider&lt;T&gt;
     /// 	</see>
     /// designed for
-    /// <see cref="PrefixTreeStrategy">PrefixTreeStrategy</see>
-    /// s.
+    /// <see cref="PrefixTreeStrategy">PrefixTreeStrategy</see>s.
     /// Note, due to the fragmented representation of Shapes in these Strategies, this implementation
-    /// can only retrieve the central
-    /// <see cref="Point">Point</see>
-    /// of the original Shapes.
+    /// can only retrieve the central <see cref="Point">Point</see> of the original Shapes.
     /// </summary>
     /// <lucene.internal></lucene.internal>
     public class PointPrefixTreeFieldCacheProvider : ShapeFieldCacheProvider<Point>
     {
-        internal readonly SpatialPrefixTree grid;
+        internal readonly SpatialPrefixTree grid; //
 
-        public PointPrefixTreeFieldCacheProvider(SpatialPrefixTree grid, string shapeField
-            , int defaultSize)
+        public PointPrefixTreeFieldCacheProvider(SpatialPrefixTree grid, string shapeField, int defaultSize)
             : base(shapeField, defaultSize)
         {
-            //
             this.grid = grid;
         }
 
-        private Cell scanCell = null;
+        private Cell scanCell = null;//re-used in readShape to save GC
 
         //re-used in readShape to save GC
         protected internal override Point ReadShape(BytesRef term)
         {
-            scanCell = grid.GetCell(term.bytes.ToByteArray(), term.offset, term.length, scanCell);
+            scanCell = grid.GetCell(term.Bytes, term.Offset, term.Length, scanCell);
             if (scanCell.Level == grid.MaxLevels && !scanCell.IsLeaf())
             {
                 return scanCell.GetCenter();

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
index 36c6177..ff83b83 100644
--- a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
@@ -25,9 +25,9 @@ using Lucene.Net.Analysis;
 using Lucene.Net.Analysis.Tokenattributes;
 using Lucene.Net.Documents;
 using Lucene.Net.Index;
-using Lucene.Net.Search.Function;
 using Lucene.Net.Spatial.Prefix.Tree;
 using Lucene.Net.Spatial.Queries;
+using Lucene.Net.Queries.Function;
 using Lucene.Net.Spatial.Util;
 using Lucene.Net.Support;
 using Spatial4n.Core.Shapes;
@@ -92,8 +92,8 @@ namespace Lucene.Net.Spatial.Prefix
     {
         protected internal readonly SpatialPrefixTree grid;
 
-        private readonly IDictionary<string, PointPrefixTreeFieldCacheProvider> provider =
-            new ConcurrentHashMap<string, PointPrefixTreeFieldCacheProvider>();
+        private readonly ConcurrentDictionary<string, PointPrefixTreeFieldCacheProvider> provider =
+            new ConcurrentDictionary<string, PointPrefixTreeFieldCacheProvider>();
 
         protected internal readonly bool simplifyIndexedCells;
 
@@ -101,11 +101,9 @@ namespace Lucene.Net.Spatial.Prefix
 
         protected internal double distErrPct = SpatialArgs.DEFAULT_DISTERRPCT;
 
-        public PrefixTreeStrategy(SpatialPrefixTree grid, string fieldName, bool simplifyIndexedCells
-            )
+        public PrefixTreeStrategy(SpatialPrefixTree grid, string fieldName, bool simplifyIndexedCells)
             : base(grid.SpatialContext, fieldName)
         {
-            // [ 0 TO 0.5 ]
             this.grid = grid;
             this.simplifyIndexedCells = simplifyIndexedCells;
         }
@@ -150,23 +148,19 @@ namespace Lucene.Net.Spatial.Prefix
             set { distErrPct = value; }
         }
 
-        public override Field[] CreateIndexableFields(Shape shape
-            )
+        public override Field[] CreateIndexableFields(Shape shape)
         {
             double distErr = SpatialArgs.CalcDistanceFromErrPct(shape, distErrPct, ctx);
             return CreateIndexableFields(shape, distErr);
         }
 
-        public virtual Field[] CreateIndexableFields(Shape shape
-            , double distErr)
+        public virtual Field[] CreateIndexableFields(Shape shape, double distErr)
         {
             int detailLevel = grid.GetLevelForDistance(distErr);
             IList<Cell> cells = grid.GetCells(shape, detailLevel, true, simplifyIndexedCells);
-            //intermediates cells
             //TODO is CellTokenStream supposed to be re-used somehow? see Uwe's comments:
             //  http://code.google.com/p/lucene-spatial-playground/issues/detail?id=4
-            Field field = new Field(FieldName, new PrefixTreeStrategy.CellTokenStream(cells
-                .GetEnumerator()), FieldType);
+            Field field = new Field(FieldName, new CellTokenStream(cells.GetEnumerator()), FieldType);
             return new Field[] { field };
         }
 
@@ -224,27 +218,10 @@ namespace Lucene.Net.Spatial.Prefix
             }
         }
 
-        public ShapeFieldCacheProvider<Point> GetCacheProvider()
+        public override ValueSource MakeDistanceValueSource(Point queryPoint, double multiplier)
         {
-            PointPrefixTreeFieldCacheProvider p;
-            if (!provider.TryGetValue(FieldName, out p) || p == null)
-            {
-                lock (this)
-                {//double checked locking idiom is okay since provider is threadsafe
-                    if (!provider.ContainsKey(FieldName))
-                    {
-                        p = new PointPrefixTreeFieldCacheProvider(grid, FieldName, defaultFieldValuesArrayLen);
-                        provider[FieldName] = p;
-                    }
-                }
-            }
-            return p;
-        }
-
-        public override ValueSource MakeDistanceValueSource(Point queryPoint)
-        {
-            var p = (PointPrefixTreeFieldCacheProvider)GetCacheProvider();
-            return new ShapeFieldCacheDistanceValueSource(ctx, p, queryPoint);
+            var p = provider.GetOrAdd(FieldName, f => new PointPrefixTreeFieldCacheProvider(grid, FieldName, defaultFieldValuesArrayLen));
+            return new ShapeFieldCacheDistanceValueSource(ctx, p, queryPoint, multiplier);
         }
 
         public virtual SpatialPrefixTree Grid

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeFilter.cs
index ce0d0d9..ad71d0c 100644
--- a/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeFilter.cs
@@ -66,9 +66,9 @@ if (!scan) {
             this.fieldName = fieldName;
             this.grid = grid;
             this.queryShape = queryShape;
-            this.prefixGridScanLevel = Math.Max(1, Math.Min(prefixGridScanLevel, grid.GetMaxLevels() - 1));
+            this.prefixGridScanLevel = Math.Max(1, Math.Min(prefixGridScanLevel, grid.MaxLevels - 1));
             this.detailLevel = detailLevel;
-            Debug.Assert(detailLevel <= grid.GetMaxLevels());
+            Debug.Assert(detailLevel <= grid.MaxLevels);
         }
 
         public override DocIdSet GetDocIdSet(Index.IndexReader reader /*, Bits acceptDocs*/)

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
index 1e0ccd0..2fcfe01 100644
--- a/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
@@ -14,6 +14,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
+using Lucene.Net.Queries.Function;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix.Tree;
 using Lucene.Net.Spatial.Queries;
@@ -35,25 +37,28 @@ namespace Lucene.Net.Spatial.Prefix
     /// <lucene.experimental></lucene.experimental>
     public class RecursivePrefixTreeStrategy : PrefixTreeStrategy
     {
+        protected readonly SpatialPrefixTree grid;
         private int prefixGridScanLevel;
 
+        /** True if only indexed points shall be supported.  See
+        *  {@link IntersectsPrefixTreeFilter#hasIndexedLeaves}. */
+        protected bool pointsOnly = false;
+
+        /** See {@link ContainsPrefixTreeFilter#multiOverlappingIndexedShapes}. */
+        protected bool multiOverlappingIndexedShapes = true;
+
         public RecursivePrefixTreeStrategy(SpatialPrefixTree grid, string fieldName)
-            : base(grid, fieldName, true)
+            : base(grid, fieldName, true) //simplify indexed cells
         {
-            //simplify indexed cells
-            prefixGridScanLevel = grid.MaxLevels - 4;
+            this.grid = grid;
+            prefixGridScanLevel = grid.MaxLevels - 4;//TODO this default constant is dependent on the prefix grid size
         }
 
-        //TODO this default constant is dependent on the prefix grid size
         /// <summary>
         /// Sets the grid level [1-maxLevels] at which indexed terms are scanned brute-force
-        /// instead of by grid decomposition.
-        /// </summary>
-        /// <remarks>
-        /// Sets the grid level [1-maxLevels] at which indexed terms are scanned brute-force
-        /// instead of by grid decomposition.  By default this is maxLevels - 4.  The
+        /// instead of by grid decomposition.By default this is maxLevels - 4.  The
         /// final level, maxLevels, is always scanned.
-        /// </remarks>
+        /// </summary>
         public virtual int PrefixGridScanLevel
         {
             set
@@ -77,27 +82,22 @@ namespace Lucene.Net.Spatial.Prefix
             }
             Shape shape = args.Shape;
             int detailLevel = grid.GetLevelForDistance(args.ResolveDistErr(ctx, distErrPct));
-            bool hasIndexedLeaves = true;
-            if (op == SpatialOperation.Intersects)
+
+        
+            if (pointsOnly || op == SpatialOperation.Intersects)
+            {
+                return new IntersectsPrefixTreeFilter(
+                    shape, FieldName, grid, detailLevel, prefixGridScanLevel, !pointsOnly);
+            }
+            else if (op == SpatialOperation.IsWithin)
             {
-                return new IntersectsPrefixTreeFilter(shape, FieldName, grid, detailLevel, prefixGridScanLevel
-                                                      , hasIndexedLeaves);
+                return new WithinPrefixTreeFilter(shape, FieldName, grid, detailLevel, prefixGridScanLevel
+                    , -1); //-1 flag is slower but ensures correct results
             }
-            else
+            else if (op == SpatialOperation.Contains)
             {
-                if (op == SpatialOperation.IsWithin)
-                {
-                    return new WithinPrefixTreeFilter(shape, FieldName, grid, detailLevel, prefixGridScanLevel
-                                                      , -1);
-                }
-                else
-                {
-                    //-1 flag is slower but ensures correct results
-                    if (op == SpatialOperation.Contains)
-                    {
-                        return new ContainsPrefixTreeFilter(shape, FieldName, grid, detailLevel);
-                    }
-                }
+                return new ContainsPrefixTreeFilter(shape, FieldName, grid, detailLevel, 
+                    multiOverlappingIndexedShapes);
             }
             throw new UnsupportedSpatialOperation(op);
         }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
index 20cbf5f..b8753e3 100644
--- a/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 using System.Collections.Generic;
+using Lucene.Net.Queries;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix.Tree;
 using Lucene.Net.Spatial.Queries;
@@ -31,7 +32,7 @@ namespace Lucene.Net.Spatial.Prefix
     /// 	</see>
     /// of all the cells from
     /// <see cref="Lucene.Net.Spatial.Prefix.Tree.SpatialPrefixTree.GetCells(Shape, int, bool, bool)
-    /// 	">Lucene.Net.Spatial.Prefix.Tree.SpatialPrefixTree.GetCells(Shape, int, bool, bool)
+    /// 	"SpatialPrefixTree.GetCells(Shape, int, bool, bool)
     /// 	</see>
     /// . It only supports the search of indexed Point shapes.
     /// <p/>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs b/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
index f572f2c..319815a 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
@@ -118,7 +118,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             //bytes = Encoding.UTF8.GetBytes(token);
             BytesRef utf8Result = new BytesRef(token.Length);
             UnicodeUtil.UTF16toUTF8(token.ToCharArray(), 0, token.Length, utf8Result);
-            bytes = utf8Result.bytes.ToByteArray();
+            bytes = utf8Result.Bytes;
 
             b_off = 0;
             b_len = bytes.Length;
@@ -192,7 +192,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
                 //bytes = Encoding.UTF8.GetBytes(token);
                 BytesRef utf8Result = new BytesRef(token.Length);
                 UnicodeUtil.UTF16toUTF8(token.ToCharArray(), 0, token.Length, utf8Result);
-                bytes = utf8Result.bytes.ToByteArray();
+                bytes = utf8Result.Bytes;
                 b_off = 0;
                 b_len = bytes.Length;
             }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Prefix/Tree/Node.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/Node.cs b/src/Lucene.Net.Spatial/Prefix/Tree/Node.cs
index bb05720..ccff798 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/Node.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/Node.cs
@@ -65,7 +65,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
 
         private void b_fixLeaf()
         {
-            if (GetLevel() == spatialPrefixTree.GetMaxLevels())
+            if (GetLevel() == spatialPrefixTree.MaxLevels)
             {
                 SetLeaf();
             }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
index 032554a..07f8f9e 100644
--- a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
@@ -149,7 +149,7 @@ namespace Lucene.Net.Spatial.Prefix
         }
 
         /// <exception cref="System.IO.IOException"></exception>
-        public override DocIdSet GetDocIdSet(AtomicReaderContext context, IBits acceptDocs
+        public override DocIdSet GetDocIdSet(AtomicReaderContext context, Bits acceptDocs
             )
         {
             return new _VisitorTemplate_121(this, context, acceptDocs, true).GetDocIdSet();
@@ -167,7 +167,7 @@ namespace Lucene.Net.Spatial.Prefix
             private SpatialRelation visitRelation;
 
             public _VisitorTemplate_121(WithinPrefixTreeFilter _enclosing, AtomicReaderContext
-                                                                               baseArg1, IBits baseArg2, bool baseArg3)
+                                                                               baseArg1, Bits baseArg2, bool baseArg3)
                 : base(_enclosing, baseArg1, baseArg2, baseArg3)
             {
                 this._enclosing = _enclosing;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/SpatialStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/SpatialStrategy.cs b/src/Lucene.Net.Spatial/SpatialStrategy.cs
index 0b283c5..2fb8689 100644
--- a/src/Lucene.Net.Spatial/SpatialStrategy.cs
+++ b/src/Lucene.Net.Spatial/SpatialStrategy.cs
@@ -17,11 +17,10 @@
 
 using System;
 using Lucene.Net.Documents;
+using Lucene.Net.Queries.Function;
+using Lucene.Net.Queries.Function.ValueSources;
 using Lucene.Net.Search;
-using Lucene.Net.Search.Function;
-using Lucene.Net.Search.Function.ValueSources;
 using Lucene.Net.Spatial.Queries;
-using Lucene.Net.Spatial.Util;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Shapes;
 
@@ -85,9 +84,15 @@ namespace Lucene.Net.Spatial
         /// <returns>Not null nor will it have null elements.</returns>
         public abstract Field[] CreateIndexableFields(Shape shape);
 
-        public Field CreateStoredField(Shape shape)
+        /// <summary>
+        /// See {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point, double)} called with
+        /// a multiplier of 1.0 (i.e. units of degrees).
+        /// </summary>
+        /// <param name="queryPoint"></param>
+        /// <returns></returns>
+        public ValueSource MakeDistanceValueSource(Point queryPoint)
         {
-            return new Field(FieldName, ctx.ToString(shape), Field.Store.YES, Field.Index.NOT_ANALYZED_NO_NORMS, Field.TermVector.NO);
+            return MakeDistanceValueSource(queryPoint, 1.0);
         }
 
         /// <summary>
@@ -95,7 +100,7 @@ namespace Lucene.Net.Spatial
         /// indexed shape and {@code queryPoint}.  If there are multiple indexed shapes
         /// then the closest one is chosen.
         /// </summary>
-        public abstract ValueSource MakeDistanceValueSource(Point queryPoint);
+        public abstract ValueSource MakeDistanceValueSource(Point queryPoint, double multiplier);
 
         /// <summary>
         /// Make a (ConstantScore) Query based principally on {@link org.apache.lucene.spatial.query.SpatialOperation}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Util/Bits.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/Bits.cs b/src/Lucene.Net.Spatial/Util/Bits.cs
deleted file mode 100644
index f0040ee..0000000
--- a/src/Lucene.Net.Spatial/Util/Bits.cs
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-namespace Lucene.Net.Spatial.Util
-{
-    /// <summary>
-    /// Interface for Bitset-like structures.
-    /// </summary>
-    public interface IBits
-    {
-        bool Get(int index);
-        int Length();
-    }
-
-    /// <summary>
-    /// Empty implementation, basically just so we can provide EMPTY_ARRAY
-    /// </summary>
-    public abstract class Bits : IBits
-    {
-        public static readonly Bits[] EMPTY_ARRAY = new Bits[0];
-
-        public virtual bool Get(int index)
-        {
-            throw new System.NotImplementedException();
-        }
-
-        public virtual int Length()
-        {
-            throw new System.NotImplementedException();
-        }
-    }
-
-    /// <summary>
-    /// Bits impl of the specified length with all bits set.
-    /// </summary>
-    public class MatchAllBits : Bits
-    {
-        private readonly int len;
-
-        public MatchAllBits(int len)
-        {
-            this.len = len;
-        }
-
-        public override bool Get(int index)
-        {
-            return true;
-        }
-
-        public override int Length()
-        {
-            return len;
-        }
-    }
-
-    /// <summary>
-    /// Bits impl of the specified length with no bits set. 
-    /// </summary>
-    public class MatchNoBits : Bits
-    {
-        private readonly int len;
-
-        public MatchNoBits(int len)
-        {
-            this.len = len;
-        }
-
-        public override bool Get(int index)
-        {
-            return false;
-        }
-
-        public override int Length()
-        {
-            return len;
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Util/CachingDoubleValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/CachingDoubleValueSource.cs b/src/Lucene.Net.Spatial/Util/CachingDoubleValueSource.cs
index b6f9c17..793e10f 100644
--- a/src/Lucene.Net.Spatial/Util/CachingDoubleValueSource.cs
+++ b/src/Lucene.Net.Spatial/Util/CachingDoubleValueSource.cs
@@ -15,21 +15,28 @@
  * limitations under the License.
  */
 
+using System.Collections;
 using System.Collections.Generic;
 using Lucene.Net.Index;
-using Lucene.Net.Search.Function;
+using Lucene.Net.Queries.Function;
+using Lucene.Net.Support;
 
 namespace Lucene.Net.Spatial.Util
 {
+    /// <summary>
+    /// Caches the doubleVal of another value source in a HashMap
+    /// so that it is computed only once.
+    /// @lucene.internal
+    /// </summary>
     public class CachingDoubleValueSource : ValueSource
     {
-        protected readonly Dictionary<int, double> cache;
+        protected readonly IDictionary<int, double> cache;
         protected readonly ValueSource source;
 
         public CachingDoubleValueSource(ValueSource source)
         {
             this.source = source;
-            cache = new Dictionary<int, double>();
+            cache = new HashMap<int, double>();
         }
 
         public override string Description
@@ -37,9 +44,9 @@ namespace Lucene.Net.Spatial.Util
             get { return "Cached[" + source.Description + "]"; }
         }
 
-        public override FunctionValues GetValues(IDictionary<object, object> context, AtomicReaderContext readerContext)
+        public override FunctionValues GetValues(IDictionary context, AtomicReaderContext readerContext)
         {
-            int @base = readerContext.docBase;
+            int @base = readerContext.DocBase;
             FunctionValues vals = source.GetValues(context, readerContext);
             return new CachingDoubleFunctionValue(@base, vals, cache);
         }
@@ -65,11 +72,11 @@ namespace Lucene.Net.Spatial.Util
 
         public class CachingDoubleFunctionValue : FunctionValues
         {
-            private readonly Dictionary<int, double> cache;
+            private readonly IDictionary<int, double> cache;
             private readonly int docBase;
             private readonly FunctionValues values;
 
-            public CachingDoubleFunctionValue(int docBase, FunctionValues vals, Dictionary<int, double> cache)
+            public CachingDoubleFunctionValue(int docBase, FunctionValues vals, IDictionary<int, double> cache)
             {
                 this.docBase = docBase;
                 values = vals;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Util/CompatibilityExtensions.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/CompatibilityExtensions.cs b/src/Lucene.Net.Spatial/Util/CompatibilityExtensions.cs
deleted file mode 100644
index be9b185..0000000
--- a/src/Lucene.Net.Spatial/Util/CompatibilityExtensions.cs
+++ /dev/null
@@ -1,173 +0,0 @@
-\ufeff/*
- * 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;
-#if NET35
-using Lucene.Net.Support.Compatibility;
-#else
-using System.Collections.Concurrent;
-#endif
-using System.Diagnostics;
-using System.Linq;
-using Lucene.Net.Analysis.Tokenattributes;
-using Lucene.Net.Index;
-using Lucene.Net.Search;
-using Lucene.Net.Util;
-
-namespace Lucene.Net.Spatial.Util
-{
-    public static class CompatibilityExtensions
-    {
-        private static readonly ConcurrentDictionary<string, IBits> _docsWithFieldCache = new ConcurrentDictionary<string, IBits>();
-
-        /// <summary> <p/>
-        /// EXPERT: Instructs the FieldCache to forcibly expunge all entries 
-        /// from the underlying caches.  This is intended only to be used for 
-        /// test methods as a way to ensure a known base state of the Cache 
-        /// (with out needing to rely on GC to free WeakReferences).  
-        /// It should not be relied on for "Cache maintenance" in general 
-        /// application code.
-        /// <p/>
-        /// <p/>
-        /// <b>EXPERIMENTAL API:</b> This API is considered extremely advanced 
-        /// and experimental.  It may be removed or altered w/o warning in future 
-        /// releases 
-        /// of Lucene.
-        /// <p/>
-        /// </summary>
-        public static void PurgeSpatialCaches(this IFieldCache fc)
-        {
-            _docsWithFieldCache.Clear();
-        }
-
-
-        public static byte[] ToByteArray(this sbyte[] sbytes)
-        {
-            return sbytes.Select(Convert.ToByte).ToArray();
-        }
-
-        /* table of number of leading zeros in a byte */
-
-        public static readonly byte[] nlzTable =
-            {
-                8, 7, 6, 6, 5, 5, 5, 5, 4, 4, 4, 4, 4, 4, 4, 4, 3, 3, 3, 3, 3, 3, 3, 3
-                , 3, 3, 3, 3, 3, 3, 3, 3, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2,
-                2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1
-                , 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1,
-                1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1
-                , 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
-                0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0
-                , 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
-                0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0
-                , 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
-                0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0
-            };
-        public static sbyte[] ToSByteArray(this byte[] bytes)
-        {
-            return bytes.Select(Convert.ToSByte).ToArray();
-        }
-
-        /// <summary>
-        /// Returns the number of leading zero bits.
-        /// </summary>
-        /// <param name="x"></param>
-        /// <returns></returns>
-        public static int BitUtilNlz(long x)
-        {
-            int n = 0;
-            // do the first step as a long
-            var y = (int)((ulong)x >> 32);
-            if (y == 0) { n += 32; y = (int)(x); }
-            if ((y & 0xFFFF0000) == 0) { n += 16; y <<= 16; }
-            if ((y & 0xFF000000) == 0) { n += 8; y <<= 8; }
-            return n + nlzTable[(uint)y >> 24];
-            /* implementation without table:
-              if ((y & 0xF0000000) == 0) { n+=4; y<<=4; }
-              if ((y & 0xC0000000) == 0) { n+=2; y<<=2; }
-              if ((y & 0x80000000) == 0) { n+=1; y<<=1; }
-              if ((y & 0x80000000) == 0) { n+=1;}
-              return n;
-             */
-        }
-    }
-
-    public static class Arrays
-    {
-        public static void Fill<T>(T[] array, int fromIndex, int toIndex, T value)
-        {
-            if (array == null)
-            {
-                throw new ArgumentNullException("array");
-            }
-            if (fromIndex < 0 || fromIndex > toIndex)
-            {
-                throw new ArgumentOutOfRangeException("fromIndex");
-            }
-            if (toIndex > array.Length)
-            {
-                throw new ArgumentOutOfRangeException("toIndex");
-            }
-            for (var i = fromIndex; i < toIndex; i++)
-            {
-                array[i] = value;
-            }
-        }
-    }
-
-    /// <summary>
-    /// Expert: Every composite-key in the internal cache is of this type.
-    /// </summary>
-    internal class Entry
-    {
-        internal readonly String field;        // which Fieldable
-        internal readonly Object custom;       // which custom comparator or parser
-
-        /* Creates one of these objects for a custom comparator/parser. */
-        public Entry(String field, Object custom)
-        {
-            this.field = field;
-            this.custom = custom;
-        }
-
-        /* Two of these are equal iff they reference the same field and type. */
-        public override bool Equals(Object o)
-        {
-            var other = o as Entry;
-            if (other != null)
-            {
-                if (other.field.Equals(field))
-                {
-                    if (other.custom == null)
-                    {
-                        if (custom == null) return true;
-                    }
-                    else if (other.custom.Equals(custom))
-                    {
-                        return true;
-                    }
-                }
-            }
-            return false;
-        }
-
-        /* Composes a hashcode based on the field and type. */
-        public override int GetHashCode()
-        {
-            return field.GetHashCode() ^ (custom == null ? 0 : custom.GetHashCode());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Util/FixedBitSet.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/FixedBitSet.cs b/src/Lucene.Net.Spatial/Util/FixedBitSet.cs
deleted file mode 100644
index bfc5c45..0000000
--- a/src/Lucene.Net.Spatial/Util/FixedBitSet.cs
+++ /dev/null
@@ -1,454 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-using System;
-using System.Collections;
-using Lucene.Net.Search;
-using Lucene.Net.Util;
-
-namespace Lucene.Net.Spatial.Util
-{
-    /* BitSet of fixed length (numBits), backed by accessible
- *  ({@link #getBits}) long[], accessed with an int index,
- *  implementing Bits and DocIdSet.  Unlike {@link
- *  OpenBitSet} this bit set does not auto-expand, cannot
- *  handle long index, and does not have fastXX/XX variants
- *  (just X).
- *
- * @lucene.internal
- **/
-    public class FixedBitSet : DocIdSet, IBits
-    {
-        private readonly BitArray bits;
-
-        /// <summary>
-        /// returns the number of 64 bit words it would take to hold numBits
-        /// </summary>
-        /// <param name="numBits"></param>
-        /// <returns></returns>
-        public static int bits2words(int numBits)
-        {
-            var numLong = (int)((uint)numBits >> 6);
-            if ((numBits & 63) != 0)
-            {
-                numLong++;
-            }
-            return numLong;
-        }
-
-        public FixedBitSet(int numBits)
-        {
-            bits = new BitArray(numBits);
-        }
-
-        /// <summary>
-        /// Makes full copy.
-        /// </summary>
-        /// <param name="other"></param>
-        public FixedBitSet(FixedBitSet other)
-        {
-            bits = new BitArray(other.bits);
-        }
-
-        public IBits Bits()
-        {
-            return this;
-        }
-
-        public int Length()
-        {
-            return bits.Length;
-        }
-
-        public override bool IsCacheable
-        {
-            get { return true; }
-        }
-
-        /// <summary>
-        /// Returns number of set bits.  NOTE: this visits every
-        /// long in the backing bits array, and the result is not
-        /// internally cached!
-        /// </summary>
-        /// <returns></returns>
-        public int Cardinality()
-        {
-            int ret = 0;
-            for (var i = 0; i < bits.Length; i++)
-            {
-                if (bits[i]) ret++;
-            }
-            return ret;
-        }
-
-        public bool Get(int index)
-        {
-            return bits[index];
-        }
-
-        public void Set(int index)
-        {
-            bits.Set(index, true);
-        }
-
-        public bool GetAndSet(int index)
-        {
-            var ret = bits[index];
-            bits.Set(index, true);
-            return ret;
-        }
-
-        public void Clear(int index)
-        {
-            bits.Set(index, false);
-        }
-
-        public bool GetAndClear(int index)
-        {
-            var ret = bits[index];
-            bits.Set(index, false);
-            return ret;
-        }
-
-        /// <summary>
-        /// Returns the index of the first set bit starting at the index specified.
-        /// -1 is returned if there are no more set bits.
-        /// </summary>
-        /// <param name="index"></param>
-        /// <returns></returns>
-        public int NextSetBit(int index)
-        {
-            if (index >= bits.Length || index < 0)
-                throw new ArgumentException("Invalid index", "index");
-
-            for (var i = index; i < bits.Length; i++)
-            {
-                if (bits[i]) return i;
-            }
-
-            return -1;
-        }
-
-        /* Returns the index of the last set bit before or on the index specified.
-         *  -1 is returned if there are no more set bits.
-         */
-        public int PrevSetBit(int index)
-        {
-            if (index >= bits.Length || index < 0)
-                throw new ArgumentException("Invalid index", "index");
-
-            for (var i = index; i >= 0; i--)
-            {
-                if (bits[i]) return i;
-            }
-
-            return -1;
-        }
-
-        /* Does in-place OR of the bits provided by the
-         *  iterator. */
-        //public void Or(DocIdSetIterator iter)
-        //{
-        //    if (iter is OpenBitSetIterator && iter.DocID() == -1)
-        //    {
-        //        var obs = (OpenBitSetIterator)iter;
-        //        Or(obs.arr, obs.words);
-        //        // advance after last doc that would be accepted if standard
-        //        // iteration is used (to exhaust it):
-        //        obs.Advance(bits.Length);
-        //    }
-        //    else
-        //    {
-        //        int doc;
-        //        while ((doc = iter.NextDoc()) < bits.Length)
-        //        {
-        //            Set(doc);
-        //        }
-        //    }
-        //}
-
-        /* this = this OR other */
-        public void Or(FixedBitSet other)
-        {
-            Or(other.bits, other.bits.Length);
-        }
-
-        private void Or(BitArray otherArr, int otherLen)
-        {
-            var thisArr = this.bits;
-            int pos = Math.Min(thisArr.Length, otherLen);
-            while (--pos >= 0)
-            {
-                thisArr[pos] |= otherArr[pos];
-            }
-        }
-
-        /* Does in-place AND of the bits provided by the
-         *  iterator. */
-        //public void And(DocIdSetIterator iter)
-        //{
-        //    if (iter is OpenBitSetIterator && iter.DocID() == -1)
-        //    {
-        //        var obs = (OpenBitSetIterator)iter;
-        //        And(obs.arr, obs.words);
-        //        // advance after last doc that would be accepted if standard
-        //        // iteration is used (to exhaust it):
-        //        obs.Advance(bits.Length);
-        //    }
-        //    else
-        //    {
-        //        if (bits.Length == 0) return;
-        //        int disiDoc, bitSetDoc = NextSetBit(0);
-        //        while (bitSetDoc != -1 && (disiDoc = iter.Advance(bitSetDoc)) < bits.Length)
-        //        {
-        //            Clear(bitSetDoc, disiDoc);
-        //            disiDoc++;
-        //            bitSetDoc = (disiDoc < bits.Length) ? NextSetBit(disiDoc) : -1;
-        //        }
-        //        if (bitSetDoc != -1)
-        //        {
-        //            Clear(bitSetDoc, bits.Length);
-        //        }
-        //    }
-        //}
-
-        /* this = this AND other */
-        public void And(FixedBitSet other)
-        {
-            And(other.bits, other.bits.Length);
-        }
-
-        private void And(BitArray otherArr, int otherLen)
-        {
-            var thisArr = this.bits;
-            int pos = Math.Min(thisArr.Length, otherLen);
-            while (--pos >= 0)
-            {
-                thisArr[pos] &= otherArr[pos];
-            }
-            if (thisArr.Length > otherLen)
-            {
-                for (var i = otherLen; i < thisArr.Length; i++)
-                {
-                    thisArr[i] = false;
-                }
-            }
-        }
-
-        /* Does in-place AND NOT of the bits provided by the
-         *  iterator. */
-        //public void AndNot(DocIdSetIterator iter)
-        //{
-        //    var obs = iter as OpenBitSetIterator;
-        //    if (obs != null && iter.DocID() == -1)
-        //    {
-        //        AndNot(obs.arr, obs.words);
-        //        // advance after last doc that would be accepted if standard
-        //        // iteration is used (to exhaust it):
-        //        obs.Advance(bits.Length);
-        //    }
-        //    else
-        //    {
-        //        int doc;
-        //        while ((doc = iter.NextDoc()) < bits.Length)
-        //        {
-        //            Clear(doc);
-        //        }
-        //    }
-        //}
-
-        /* this = this AND NOT other */
-        public void AndNot(FixedBitSet other)
-        {
-            AndNot(other.bits, other.bits.Length);
-        }
-
-        private void AndNot(BitArray otherArr, int otherLen)
-        {
-            var thisArr = this.bits;
-            int pos = Math.Min(thisArr.Length, otherLen);
-            while (--pos >= 0)
-            {
-                thisArr[pos] &= !otherArr[pos];
-            }
-        }
-
-        // NOTE: no .isEmpty() here because that's trappy (ie,
-        // typically isEmpty is low cost, but this one wouldn't
-        // be)
-
-        /* Flips a range of bits
-         *
-         * @param startIndex lower index
-         * @param endIndex one-past the last bit to flip
-         */
-        //      public void Flip(int startIndex, int endIndex) {
-        //  Debug.Assert(startIndex >= 0 && startIndex < numBits);
-        //  Debug.Assert(endIndex >= 0 && endIndex <= numBits);
-        //  if (endIndex <= startIndex) {
-        //    return;
-        //  }
-
-        //  int startWord = startIndex >> 6;
-        //  int endWord = (endIndex-1) >> 6;
-
-        //  /* Grrr, java shifting wraps around so -1L>>>64 == -1
-        //   * for that reason, make sure not to use endmask if the bits to flip will
-        //   * be zero in the last word (redefine endWord to be the last changed...)
-        //  long startmask = -1L << (startIndex & 0x3f);     // example: 11111...111000
-        //  long endmask = -1L >>> (64-(endIndex & 0x3f));   // example: 00111...111111
-        //  ***/
-
-        //  long startmask = -1L << startIndex;
-        //  long endmask =  -1L >>> -endIndex;  // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
-
-        //  if (startWord == endWord) {
-        //    bits[startWord] ^= (startmask & endmask);
-        //    return;
-        //  }
-
-        //  bits[startWord] ^= startmask;
-
-        //  for (var i=startWord+1; i<endWord; i++) {
-        //    bits[i] = ~bits[i];
-        //  }
-
-        //  bits[endWord] ^= endmask;
-        //}
-
-        /* Sets a range of bits
-         *
-         * @param startIndex lower index
-         * @param endIndex one-past the last bit to set
-         */
-        public void Set(int startIndex, int endIndex)
-        {
-            // Naive implementation
-            for (int i = startIndex; i < endIndex; i++)
-            {
-                Set(i);
-            }
-        }
-
-        //      public void Set(int startIndex, int endIndex) {
-        //  Debug.Assert(startIndex >= 0 && startIndex < numBits);
-        //  Debug.Assert(endIndex >= 0 && endIndex <= numBits);
-        //  if (endIndex <= startIndex) {
-        //    return;
-        //  }
-
-        //  int startWord = startIndex >> 6;
-        //  int endWord = (endIndex-1) >> 6;
-
-        //  long startmask = -1L << startIndex;
-        //  long endmask = -1L >>> -endIndex;  // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
-
-        //  if (startWord == endWord) {
-        //    bits[startWord] |= (startmask & endmask);
-        //    return;
-        //  }
-
-        //  bits[startWord] |= startmask;
-        //  Arrays.Fill(bits, startWord+1, endWord, -1L);
-        //  bits[endWord] |= endmask;
-        //}
-
-        /* Clears a range of bits.
-         *
-         * @param startIndex lower index
-         * @param endIndex one-past the last bit to clear
-         */
-        public void Clear(int startIndex, int endIndex)
-        {
-            for (int i = startIndex; i < endIndex; i++)
-            {
-                Clear(i);
-            }
-        }
-
-        //@Override
-        public FixedBitSet Clone()
-        {
-            return new FixedBitSet(this);
-        }
-
-        /* returns true if both sets have the same bits set */
-        public override bool Equals(Object o)
-        {
-            if (this == o)
-            {
-                return true;
-            }
-
-            var other = o as FixedBitSet;
-            if (other == null)
-            {
-                return false;
-            }
-
-            return bits.Equals(other.bits);
-        }
-
-        public override int GetHashCode()
-        {
-            return bits.GetHashCode();
-        }
-
-        public override DocIdSetIterator Iterator()
-        {
-            return new FixedBitSetIterator(this);
-        }
-
-        /// <summary>
-        /// A FixedBitSet Iterator implementation
-        /// </summary>
-        public class FixedBitSetIterator : DocIdSetIterator
-        {
-            private int curDocId = -1;
-            private readonly IEnumerator enumerator;
-
-            public FixedBitSetIterator(FixedBitSet bitset)
-            {
-                enumerator = bitset.bits.GetEnumerator();
-            }
-
-            public override int DocID()
-            {
-                return curDocId;
-            }
-
-            public override int NextDoc()
-            {
-                while (enumerator.MoveNext())
-                {
-                    ++curDocId;
-                    if ((bool)enumerator.Current) return curDocId;
-                }
-                return curDocId = NO_MORE_DOCS;
-            }
-
-            public override int Advance(int target)
-            {
-                int doc;
-                while ((doc = NextDoc()) < target)
-                {
-                }
-                return doc;
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Util/FunctionQuery.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/FunctionQuery.cs b/src/Lucene.Net.Spatial/Util/FunctionQuery.cs
deleted file mode 100644
index 8d5d1c7..0000000
--- a/src/Lucene.Net.Spatial/Util/FunctionQuery.cs
+++ /dev/null
@@ -1,215 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-using System;
-using Lucene.Net.Index;
-using Lucene.Net.Search;
-using Lucene.Net.Search.Function;
-
-namespace Lucene.Net.Spatial.Util
-{
-    /// <summary>
-    /// Port of Solr's FunctionQuery (v1.4)
-    /// 
-    /// Returns a score for each document based on a ValueSource,
-    /// often some function of the value of a field.
-    /// 
-    /// <b>Note: This API is experimental and may change in non backward-compatible ways in the future</b>
-    /// </summary>
-    public class FunctionQuery : Query
-    {
-        protected readonly ValueSource func;
-
-        public FunctionQuery(ValueSource func)
-        {
-            this.func = func;
-        }
-
-        /// <summary>
-        /// 
-        /// </summary>
-        /// <returns>The associated ValueSource</returns>
-        public ValueSource GetValueSource()
-        {
-            return func;
-        }
-
-        public override Query Rewrite(Index.IndexReader reader)
-        {
-            return this;
-        }
-
-        public override void ExtractTerms(System.Collections.Generic.ISet<Term> terms)
-        {
-            //base.ExtractTerms(terms);
-        }
-
-        protected class FunctionWeight : Weight
-        {
-            protected Searcher searcher;
-            protected float queryNorm;
-            protected float queryWeight;
-            protected readonly FunctionQuery enclosingInstance;
-
-            public FunctionWeight(Searcher searcher, FunctionQuery q)
-            {
-                enclosingInstance = q;
-                this.searcher = searcher;
-                //q.func.CreateWeight(searcher);
-            }
-
-            internal float GetQueryNorm()
-            {
-                return queryNorm;
-            }
-
-            public override Query Query
-            {
-                get { return enclosingInstance; }
-            }
-
-            public override float Value
-            {
-                get { return queryWeight; }
-            }
-
-            public override float GetSumOfSquaredWeights()
-            {
-                queryWeight = enclosingInstance.Boost;
-                return queryWeight * queryWeight;
-            }
-
-            public override void Normalize(float norm)
-            {
-                this.queryNorm = norm;
-                queryWeight *= this.queryNorm;
-            }
-
-            public override Scorer Scorer(IndexReader reader, bool scoreDocsInOrder, bool topScorer)
-            {
-                return new AllScorer(enclosingInstance.GetSimilarity(searcher), reader, this);
-            }
-
-            public override Explanation Explain(IndexReader reader, int doc)
-            {
-                return ((AllScorer)Scorer(reader, true, true)).Explain(doc);
-            }
-        }
-
-        protected class AllScorer : Scorer
-        {
-            readonly IndexReader reader;
-            readonly FunctionWeight weight;
-            readonly int maxDoc;
-            readonly float qWeight;
-            int doc = -1;
-            readonly DocValues vals;
-            readonly bool hasDeletions;
-
-            public AllScorer(Similarity similarity, IndexReader reader, FunctionWeight w)
-                : base(similarity)
-            {
-                this.weight = w;
-                this.qWeight = w.Value;
-                this.reader = reader;
-                this.maxDoc = reader.MaxDoc;
-                this.hasDeletions = reader.HasDeletions;
-                vals = ((FunctionQuery)w.Query).func.GetValues(reader);
-            }
-
-            public override int DocID()
-            {
-                return doc;
-            }
-
-            // instead of matching all docs, we could also embed a query.
-            // the score could either ignore the subscore, or boost it.
-            // Containment:  floatline(foo:myTerm, "myFloatField", 1.0, 0.0f)
-            // Boost:        foo:myTerm^floatline("myFloatField",1.0,0.0f)
-            public override int NextDoc()
-            {
-                for (; ; )
-                {
-                    ++doc;
-                    if (doc >= maxDoc)
-                    {
-                        return doc = NO_MORE_DOCS;
-                    }
-                    if (hasDeletions && reader.IsDeleted(doc)) continue;
-                    return doc;
-                }
-            }
-
-            public override int Advance(int target)
-            {
-                // this will work even if target==NO_MORE_DOCS
-                doc = target - 1;
-                return NextDoc();
-            }
-
-            public override float Score()
-            {
-                float score = qWeight * vals.FloatVal(doc);
-
-                // Current Lucene priority queues can't handle NaN and -Infinity, so
-                // map to -Float.MAX_VALUE. This conditional handles both -infinity
-                // and NaN since comparisons with NaN are always false.
-                return score > float.NegativeInfinity ? score : -float.MaxValue;
-            }
-
-            public /*override*/ Explanation Explain(int doc)
-            {
-                float sc = qWeight * vals.FloatVal(doc);
-
-                Explanation result = new ComplexExplanation
-                  (true, sc, "FunctionQuery(" + ((FunctionQuery)weight.Query).func + "), product of:");
-
-                result.AddDetail(vals.Explain(doc));
-                result.AddDetail(new Explanation(weight.Query.Boost, "boost"));
-                result.AddDetail(new Explanation(weight.GetQueryNorm(), "queryNorm"));
-                return result;
-            }
-        }
-
-        public override Weight CreateWeight(Searcher searcher)
-        {
-            return new FunctionQuery.FunctionWeight(searcher, this);
-        }
-
-        public override string ToString(string field)
-        {
-            float boost = Boost;
-            return (boost != 1.0 ? "(" : "") + func.ToString()
-                    + (boost == 1.0 ? "" : ")^" + boost);
-        }
-
-        public override bool Equals(object o)
-        {
-            var other = o as FunctionQuery;
-
-            if (other == null) return false;
-
-            return this.Boost == other.Boost && this.func.Equals(other.func);
-        }
-
-        public override int GetHashCode()
-        {
-            return (int)(func.GetHashCode() * 31 + BitConverter.DoubleToInt64Bits(Boost));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Util/ReciprocalFloatFunction.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ReciprocalFloatFunction.cs b/src/Lucene.Net.Spatial/Util/ReciprocalFloatFunction.cs
deleted file mode 100644
index 010f628..0000000
--- a/src/Lucene.Net.Spatial/Util/ReciprocalFloatFunction.cs
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-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()
-        {
-            int h = (int)BitConverter.DoubleToInt64Bits(a) + (int)BitConverter.DoubleToInt64Bits(m);
-            h ^= (h << 13) | (int)((uint)h >> 20);
-            return h + ((int)BitConverter.DoubleToInt64Bits(b)) + source.GetHashCode();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs b/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
index b487d0c..d1bcb9e 100644
--- a/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
+++ b/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
@@ -16,9 +16,10 @@
  */
 
 using System;
+using System.Collections;
 using System.Collections.Generic;
 using Lucene.Net.Index;
-using Lucene.Net.Search.Function;
+using Lucene.Net.Queries.Function;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Distance;
 using Spatial4n.Core.Shapes;
@@ -33,12 +34,15 @@ namespace Lucene.Net.Spatial.Util
         private readonly ShapeFieldCacheProvider<Point> provider;
         private readonly SpatialContext ctx;
         private readonly Point from;
+        private readonly 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;
         }
 
         public class CachedDistanceFunctionValue : FunctionValues
@@ -56,7 +60,7 @@ namespace Lucene.Net.Spatial.Util
 
                 from = enclosingInstance.from;
                 calculator = enclosingInstance.ctx.GetDistCalc();
-                nullValue = (enclosingInstance.ctx.IsGeo() ? 180 : double.MaxValue);
+                nullValue = (enclosingInstance.ctx.IsGeo() ? 180 * enclosingInstance.multiplier : double.MaxValue);
             }
 
             public override float FloatVal(int doc)
@@ -74,7 +78,7 @@ namespace Lucene.Net.Spatial.Util
                     {
                         v = Math.Min(v, calculator.Distance(from, vals[i]));
                     }
-                    return v;
+                    return v * enclosingInstance.multiplier;
                 }
                 return nullValue;
             }
@@ -93,7 +97,7 @@ namespace Lucene.Net.Spatial.Util
             }
         }
 
-        public override FunctionValues GetValues(IDictionary<object, object> context, AtomicReaderContext readerContext)
+        public override FunctionValues GetValues(IDictionary context, AtomicReaderContext readerContext)
         {
             return new CachedDistanceFunctionValue(readerContext.AtomicReader, this);
         }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Util/TermsFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/TermsFilter.cs b/src/Lucene.Net.Spatial/Util/TermsFilter.cs
deleted file mode 100644
index 4acfcef..0000000
--- a/src/Lucene.Net.Spatial/Util/TermsFilter.cs
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-using System;
-using System.Collections.Generic;
-using System.Diagnostics;
-using Lucene.Net.Index;
-using Lucene.Net.Search;
-
-namespace Lucene.Net.Spatial.Util
-{
-    /// <summary>
-    /// Constructs a filter for docs matching any of the terms added to this class.
-    /// Unlike a RangeFilter this can be used for filtering on multiple terms that are not necessarily in
-    /// a sequence. An example might be a collection of primary keys from a database query result or perhaps
-    /// a choice of "category" labels picked by the end user. As a filter, this is much faster than the
-    /// equivalent query (a BooleanQuery with many "should" TermQueries)
-    /// </summary>
-    public class TermsFilter : Filter
-    {
-        private readonly SortedSet<Term> terms = new SortedSet<Term>();
-
-        /// <summary>
-        /// Adds a term to the list of acceptable terms
-        /// </summary>
-        /// <param name="term"></param>
-        public void AddTerm(Term term)
-        {
-            terms.Add(term);
-        }
-
-        public override DocIdSet GetDocIdSet(IndexReader reader)
-        {
-            var result = new FixedBitSet(reader.MaxDoc);
-            var fields = reader.GetFieldNames(IndexReader.FieldOption.ALL);
-
-            if (fields == null || fields.Count == 0)
-            {
-                return result;
-            }
-
-            String lastField = null;
-            TermsEnumCompatibility termsEnum = null;
-            foreach (Term term in terms)
-            {
-                if (!term.Field.Equals(lastField))
-                {
-                    var termsC = new TermsEnumCompatibility(reader, term.Field);
-                    if (termsC.Term() == null)
-                    {
-                        return result;
-                    }
-                    termsEnum = termsC;
-                    lastField = term.Field;
-                }
-
-                if (terms != null)
-                {
-                    // TODO this check doesn't make sense, decide which variable its supposed to be for
-                    Debug.Assert(termsEnum != null);
-                    if (termsEnum.SeekCeil(term.Text) == TermsEnumCompatibility.SeekStatus.FOUND)
-                    {
-                        termsEnum.Docs(result);
-                    }
-                }
-            }
-            return result;
-        }
-
-        public override bool Equals(object obj)
-        {
-            if (this == obj)
-                return true;
-
-            if ((obj == null) || (obj.GetType() != this.GetType()))
-                return false;
-
-            var test = (TermsFilter)obj;
-            if (terms == test.terms)
-                return true;
-            if (terms == null || terms.Count != test.terms.Count)
-                return false;
-
-            var e1 = terms.GetEnumerator();
-            var e2 = test.terms.GetEnumerator();
-            while (e1.MoveNext() && e2.MoveNext())
-            {
-                if (!e1.Current.Equals(e2.Current)) return false;
-            }
-            return true;
-        }
-
-        public override int GetHashCode()
-        {
-            int hash = 9;
-            foreach (Term term in terms)
-            {
-                hash = 31 * hash + term.GetHashCode();
-            }
-            return hash;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs b/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
index 6bba8c7..bcd73a6 100644
--- a/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
+++ b/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
@@ -17,8 +17,8 @@
 
 using System;
 using Lucene.Net.Index;
+using Lucene.Net.Queries.Function;
 using Lucene.Net.Search;
-using Lucene.Net.Search.Function;
 using Lucene.Net.Util;
 
 namespace Lucene.Net.Spatial.Util
@@ -46,7 +46,7 @@ namespace Lucene.Net.Spatial.Util
             this.max = max;
         }
 
-        public override DocIdSet GetDocIdSet(AtomicReaderContext context, IBits acceptDocs)
+        public override DocIdSet GetDocIdSet(AtomicReaderContext context, Bits acceptDocs)
         {
             var values = source.GetValues(null, context);
             return new ValueSourceFilteredDocIdSet(startingFilter.GetDocIdSet(context, acceptDocs), values, this);
@@ -64,7 +64,7 @@ namespace Lucene.Net.Spatial.Util
                 this.values = values;
             }
 
-            public override bool Match(int doc)
+            protected override bool Match(int doc)
             {
                 double val = values.DoubleVal(doc);
                 return val >= enclosingFilter.min && val <= enclosingFilter.max;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs b/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
index 509146a..d54016f 100644
--- a/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
+++ b/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
@@ -15,11 +15,12 @@
  * limitations under the License.
  */
 
+using System.Collections;
 using System.Collections.Generic;
 using System.Diagnostics;
 using Lucene.Net.Index;
+using Lucene.Net.Queries.Function;
 using Lucene.Net.Search;
-using Lucene.Net.Search.Function;
 using Lucene.Net.Util;
 using Spatial4n.Core.Distance;
 using Spatial4n.Core.Shapes;
@@ -33,11 +34,13 @@ namespace Lucene.Net.Spatial.Vector
     {
         private readonly Point from;
         private readonly PointVectorStrategy strategy;
+        private readonly double multiplier;
 
-        public DistanceValueSource(PointVectorStrategy strategy, Point from)
+        public DistanceValueSource(PointVectorStrategy strategy, Point from, double multiplier)
         {
             this.strategy = strategy;
             this.from = from;
+            this.multiplier = multiplier;
         }
 
         public override string Description
@@ -45,7 +48,7 @@ namespace Lucene.Net.Spatial.Vector
             get { return "DistanceValueSource(" + strategy + ", " + from + ")"; }
         }
 
-        public override FunctionValues GetValues(IDictionary<object, object> context, AtomicReaderContext readerContext)
+        public override FunctionValues GetValues(IDictionary context, AtomicReaderContext readerContext)
         {
             return new DistanceFunctionValue(this, readerContext.AtomicReader);
         }
@@ -76,9 +79,10 @@ namespace Lucene.Net.Spatial.Vector
             private readonly DistanceValueSource enclosingInstance;
             private readonly Point from;
             private readonly double nullValue;
+            private readonly double multiplier;
 
             private readonly FieldCache.Doubles ptX, ptY;
-            private readonly IBits validX, validY;
+            private readonly Bits validX, validY;
 
             public DistanceFunctionValue(DistanceValueSource enclosingInstance, AtomicReader reader)
             {
@@ -90,8 +94,9 @@ namespace Lucene.Net.Spatial.Vector
                 validY = FieldCache.DEFAULT.GetDocsWithField(reader, enclosingInstance.strategy.FieldNameY);
 
                 from = enclosingInstance.from;
+                multiplier = enclosingInstance.multiplier;
                 calculator = enclosingInstance.strategy.SpatialContext.GetDistCalc();
-                nullValue = (enclosingInstance.strategy.SpatialContext.IsGeo() ? 180 : double.MaxValue);
+                nullValue = (enclosingInstance.strategy.SpatialContext.IsGeo() ? 180 * multiplier : double.MaxValue);
             }
 
             public override float FloatVal(int doc)
@@ -102,10 +107,10 @@ namespace Lucene.Net.Spatial.Vector
             public override double DoubleVal(int doc)
             {
                 // make sure it has minX and area
-                if (validX[doc])
+                if (validX.Get(doc))
                 {
-                    Debug.Assert(validY[doc]);
-                    return calculator.Distance(from, ptX.Get(doc), ptY.Get(doc));
+                    Debug.Assert(validY.Get(doc));
+                    return calculator.Distance(from, ptX.Get(doc), ptY.Get(doc)) * multiplier;
                 }
                 return nullValue;
             }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
index 0f1238f..aec23a0 100644
--- a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
+++ b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
@@ -17,8 +17,8 @@
 
 using System;
 using Lucene.Net.Documents;
+using Lucene.Net.Queries.Function;
 using Lucene.Net.Search;
-using Lucene.Net.Search.Function;
 using Lucene.Net.Spatial.Queries;
 using Lucene.Net.Spatial.Util;
 using Spatial4n.Core.Context;
@@ -96,9 +96,9 @@ namespace Lucene.Net.Spatial.Vector
             return f;
         }
 
-        public override ValueSource MakeDistanceValueSource(Point queryPoint)
+        public override ValueSource MakeDistanceValueSource(Point queryPoint, double multiplier)
         {
-            return new DistanceValueSource(this, queryPoint);
+            return new DistanceValueSource(this, queryPoint, multiplier);
         }
 
         public override ConstantScoreQuery MakeQuery(SpatialArgs args)
@@ -192,8 +192,8 @@ namespace Lucene.Net.Spatial.Vector
             }
             Query spatialRankingQuery = new FunctionQuery(valueSource);
             var bq = new BooleanQuery();
-            bq.Add(spatial, Occur.MUST);
-            bq.Add(spatialRankingQuery, Occur.MUST);
+            bq.Add(spatial, BooleanClause.Occur.MUST);
+            bq.Add(spatialRankingQuery, BooleanClause.Occur.MUST);
             return bq;
 
         }
@@ -216,12 +216,12 @@ namespace Lucene.Net.Spatial.Vector
         private Query MakeWithin(Rectangle bbox)
         {
             var bq = new BooleanQuery();
-            const Occur MUST = Occur.MUST;
+            const BooleanClause.Occur MUST = BooleanClause.Occur.MUST;
             if (bbox.GetCrossesDateLine())
             {
                 //use null as performance trick since no data will be beyond the world bounds
-                bq.Add(RangeQuery(fieldNameX, null /*-180*/, bbox.GetMaxX()), Occur.SHOULD);
-                bq.Add(RangeQuery(fieldNameX, bbox.GetMinX(), null /*+180*/), Occur.SHOULD);
+                bq.Add(RangeQuery(fieldNameX, null /*-180*/, bbox.GetMaxX()), BooleanClause.Occur.SHOULD);
+                bq.Add(RangeQuery(fieldNameX, bbox.GetMinX(), null /*+180*/), BooleanClause.Occur.SHOULD);
                 bq.MinimumNumberShouldMatch = 1; //must match at least one of the SHOULD
             }
             else
@@ -253,7 +253,7 @@ namespace Lucene.Net.Spatial.Vector
                 throw new InvalidOperationException("MakeDisjoint doesn't handle dateline cross");
             Query qX = RangeQuery(fieldNameX, bbox.GetMinX(), bbox.GetMaxX());
             Query qY = RangeQuery(fieldNameY, bbox.GetMinY(), bbox.GetMaxY());
-            var bq = new BooleanQuery { { qX, Occur.MUST_NOT }, { qY, Occur.MUST_NOT } };
+            var bq = new BooleanQuery { { qX, BooleanClause.Occur.MUST_NOT }, { qY, BooleanClause.Occur.MUST_NOT } };
             return bq;
         }
     }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/packages.config
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/packages.config b/src/Lucene.Net.Spatial/packages.config
new file mode 100644
index 0000000..27237fc
--- /dev/null
+++ b/src/Lucene.Net.Spatial/packages.config
@@ -0,0 +1,4 @@
+\ufeff<?xml version="1.0" encoding="utf-8"?>
+<packages>
+  <package id="Spatial4n.Core" version="0.3" targetFramework="net451" />
+</packages>
\ No newline at end of file


[02/26] lucenenet git commit: Ported over Lucene.Net.Spatial

Posted by ni...@apache.org.
Ported over Lucene.Net.Spatial


Project: http://git-wip-us.apache.org/repos/asf/lucenenet/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucenenet/commit/4a485ee4
Tree: http://git-wip-us.apache.org/repos/asf/lucenenet/tree/4a485ee4
Diff: http://git-wip-us.apache.org/repos/asf/lucenenet/diff/4a485ee4

Branch: refs/heads/master
Commit: 4a485ee473b7b6a0ed5304b4817bd5b9670f7a7a
Parents: 96d38ef
Author: Naz Junaid <na...@gmail.com>
Authored: Wed Jul 27 00:08:00 2016 +0100
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Tue Nov 15 19:17:53 2016 +0700

----------------------------------------------------------------------
 Lucene.Net.sln                                  | 247 ++++++++++
 .../Search/BitsFilteredDocIdSet.cs              |   2 +-
 src/Lucene.Net.Core/Search/FilteredDocIdSet.cs  |   2 +-
 src/Lucene.Net.Spatial/Contrib.Spatial.csproj   | 202 ---------
 src/Lucene.Net.Spatial/DisjointSpatialFilter.cs |  42 +-
 .../Lucene.Net.Spatial.csproj                   | 119 +++++
 .../Prefix/AbstractPrefixTreeFilter.cs          |   4 +-
 .../Prefix/AbstractVisitingPrefixTreeFilter.cs  |  24 +-
 .../Prefix/ContainsPrefixTreeFilter.cs          | 222 ++++-----
 .../Prefix/IntersectsPrefixTreeFilter.cs        |  15 +-
 .../Prefix/PointPrefixTreeFieldCacheProvider.cs |  17 +-
 .../Prefix/PrefixTreeStrategy.cs                |  43 +-
 .../Prefix/RecursivePrefixTreeFilter.cs         |   4 +-
 .../Prefix/RecursivePrefixTreeStrategy.cs       |  56 +--
 .../Prefix/TermQueryPrefixTreeStrategy.cs       |   3 +-
 src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs      |   4 +-
 src/Lucene.Net.Spatial/Prefix/Tree/Node.cs      |   2 +-
 .../Prefix/WithinPrefixTreeFilter.cs            |   4 +-
 src/Lucene.Net.Spatial/SpatialStrategy.cs       |  17 +-
 src/Lucene.Net.Spatial/Util/Bits.cs             |  92 ----
 .../Util/CachingDoubleValueSource.cs            |  21 +-
 .../Util/CompatibilityExtensions.cs             | 173 -------
 src/Lucene.Net.Spatial/Util/FixedBitSet.cs      | 454 -------------------
 src/Lucene.Net.Spatial/Util/FunctionQuery.cs    | 215 ---------
 .../Util/ReciprocalFloatFunction.cs             | 100 ----
 .../Util/ShapeFieldCacheDistanceValueSource.cs  |  14 +-
 src/Lucene.Net.Spatial/Util/TermsFilter.cs      | 117 -----
 .../Util/ValueSourceFilter.cs                   |   6 +-
 .../Vector/DistanceValueSource.cs               |  21 +-
 .../Vector/PointVectorStrategy.cs               |  18 +-
 src/Lucene.Net.Spatial/packages.config          |   4 +
 31 files changed, 636 insertions(+), 1628 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/Lucene.Net.sln
----------------------------------------------------------------------
diff --git a/Lucene.Net.sln b/Lucene.Net.sln
index 7591ffc..7fe1c8e 100644
--- a/Lucene.Net.sln
+++ b/Lucene.Net.sln
@@ -69,15 +69,22 @@ EndProject
 Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Lucene.Net.Tests.Analysis.Stempel", "src\Lucene.Net.Tests.Analysis.Stempel\Lucene.Net.Tests.Analysis.Stempel.csproj", "{940A6AB1-F00A-40E2-BC1A-2898EFA8C48F}"
 EndProject
 Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Lucene.Net.Tests.Grouping", "src\Lucene.Net.Tests.Grouping\Lucene.Net.Tests.Grouping.csproj", "{C2349F0D-FB66-4544-9C33-4D87F73C6004}"
+Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Lucene.Net.Spatial", "src\Lucene.Net.Spatial\Lucene.Net.Spatial.csproj", "{35C347F4-24B2-4BE5-8117-A0E3001551CE}"
 EndProject
 Global
 	GlobalSection(SolutionConfigurationPlatforms) = preSolution
 		Debug|Any CPU = Debug|Any CPU
 		Debug|Mixed Platforms = Debug|Mixed Platforms
 		Debug|x86 = Debug|x86
+		Debug35|Any CPU = Debug35|Any CPU
+		Debug35|Mixed Platforms = Debug35|Mixed Platforms
+		Debug35|x86 = Debug35|x86
 		Release|Any CPU = Release|Any CPU
 		Release|Mixed Platforms = Release|Mixed Platforms
 		Release|x86 = Release|x86
+		Release35|Any CPU = Release35|Any CPU
+		Release35|Mixed Platforms = Release35|Mixed Platforms
+		Release35|x86 = Release35|x86
 	EndGlobalSection
 	GlobalSection(ProjectConfigurationPlatforms) = postSolution
 		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Debug|Any CPU.ActiveCfg = Debug|x86
@@ -85,184 +92,394 @@ Global
 		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Debug|Mixed Platforms.Build.0 = Debug|x86
 		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Debug|x86.ActiveCfg = Debug|x86
 		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Debug|x86.Build.0 = Debug|x86
+		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Debug35|Any CPU.ActiveCfg = Release|x86
+		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Debug35|Any CPU.Build.0 = Release|x86
+		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Debug35|Mixed Platforms.ActiveCfg = Debug|x86
+		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Debug35|Mixed Platforms.Build.0 = Debug|x86
+		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Debug35|x86.ActiveCfg = Debug|x86
+		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Debug35|x86.Build.0 = Debug|x86
 		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Release|Any CPU.ActiveCfg = Release|x86
 		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Release|Mixed Platforms.ActiveCfg = Release|x86
 		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Release|Mixed Platforms.Build.0 = Release|x86
 		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Release|x86.ActiveCfg = Release|x86
 		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Release|x86.Build.0 = Release|x86
+		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Release35|Any CPU.ActiveCfg = Release|x86
+		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Release35|Any CPU.Build.0 = Release|x86
+		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Release35|Mixed Platforms.ActiveCfg = Release|x86
+		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Release35|Mixed Platforms.Build.0 = Release|x86
+		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Release35|x86.ActiveCfg = Release|x86
+		{5D4AD9BE-1FFB-41AB-9943-25737971BF57}.Release35|x86.Build.0 = Release|x86
 		{DE63DB10-975F-460D-AF85-572C17A91284}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{DE63DB10-975F-460D-AF85-572C17A91284}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{DE63DB10-975F-460D-AF85-572C17A91284}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{DE63DB10-975F-460D-AF85-572C17A91284}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{DE63DB10-975F-460D-AF85-572C17A91284}.Debug|x86.ActiveCfg = Debug|Any CPU
 		{DE63DB10-975F-460D-AF85-572C17A91284}.Debug|x86.Build.0 = Debug|Any CPU
+		{DE63DB10-975F-460D-AF85-572C17A91284}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{DE63DB10-975F-460D-AF85-572C17A91284}.Debug35|Any CPU.Build.0 = Debug|Any CPU
+		{DE63DB10-975F-460D-AF85-572C17A91284}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{DE63DB10-975F-460D-AF85-572C17A91284}.Debug35|Mixed Platforms.Build.0 = Debug|Any CPU
+		{DE63DB10-975F-460D-AF85-572C17A91284}.Debug35|x86.ActiveCfg = Debug|Any CPU
+		{DE63DB10-975F-460D-AF85-572C17A91284}.Debug35|x86.Build.0 = Debug|Any CPU
 		{DE63DB10-975F-460D-AF85-572C17A91284}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{DE63DB10-975F-460D-AF85-572C17A91284}.Release|Any CPU.Build.0 = Release|Any CPU
 		{DE63DB10-975F-460D-AF85-572C17A91284}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{DE63DB10-975F-460D-AF85-572C17A91284}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{DE63DB10-975F-460D-AF85-572C17A91284}.Release|x86.ActiveCfg = Release|Any CPU
+		{DE63DB10-975F-460D-AF85-572C17A91284}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{DE63DB10-975F-460D-AF85-572C17A91284}.Release35|Any CPU.Build.0 = Release|Any CPU
+		{DE63DB10-975F-460D-AF85-572C17A91284}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{DE63DB10-975F-460D-AF85-572C17A91284}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
+		{DE63DB10-975F-460D-AF85-572C17A91284}.Release35|x86.ActiveCfg = Release|Any CPU
+		{DE63DB10-975F-460D-AF85-572C17A91284}.Release35|x86.Build.0 = Release|Any CPU
 		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Debug|x86.ActiveCfg = Debug|Any CPU
+		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Debug35|Any CPU.Build.0 = Debug|Any CPU
+		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Debug35|Mixed Platforms.Build.0 = Debug|Any CPU
+		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Debug35|x86.ActiveCfg = Debug|Any CPU
+		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Debug35|x86.Build.0 = Debug|Any CPU
 		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Release|Any CPU.Build.0 = Release|Any CPU
 		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Release|x86.ActiveCfg = Release|Any CPU
+		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Release35|Any CPU.Build.0 = Release|Any CPU
+		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
+		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Release35|x86.ActiveCfg = Release|Any CPU
+		{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}.Release35|x86.Build.0 = Release|Any CPU
 		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Debug|x86.ActiveCfg = Debug|Any CPU
+		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Debug35|Any CPU.Build.0 = Debug|Any CPU
+		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Debug35|Mixed Platforms.Build.0 = Debug|Any CPU
+		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Debug35|x86.ActiveCfg = Debug|Any CPU
+		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Debug35|x86.Build.0 = Debug|Any CPU
 		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Release|Any CPU.Build.0 = Release|Any CPU
 		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Release|x86.ActiveCfg = Release|Any CPU
+		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Release35|Any CPU.Build.0 = Release|Any CPU
+		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
+		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Release35|x86.ActiveCfg = Release|Any CPU
+		{69D7956C-C2CC-4708-B399-A188FEC384C4}.Release35|x86.Build.0 = Release|Any CPU
 		{48F7884A-9454-4E88-8413-9D35992CB440}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{48F7884A-9454-4E88-8413-9D35992CB440}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{48F7884A-9454-4E88-8413-9D35992CB440}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{48F7884A-9454-4E88-8413-9D35992CB440}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{48F7884A-9454-4E88-8413-9D35992CB440}.Debug|x86.ActiveCfg = Debug|Any CPU
+		{48F7884A-9454-4E88-8413-9D35992CB440}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{48F7884A-9454-4E88-8413-9D35992CB440}.Debug35|Any CPU.Build.0 = Debug|Any CPU
+		{48F7884A-9454-4E88-8413-9D35992CB440}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{48F7884A-9454-4E88-8413-9D35992CB440}.Debug35|Mixed Platforms.Build.0 = Debug|Any CPU
+		{48F7884A-9454-4E88-8413-9D35992CB440}.Debug35|x86.ActiveCfg = Debug|Any CPU
+		{48F7884A-9454-4E88-8413-9D35992CB440}.Debug35|x86.Build.0 = Debug|Any CPU
 		{48F7884A-9454-4E88-8413-9D35992CB440}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{48F7884A-9454-4E88-8413-9D35992CB440}.Release|Any CPU.Build.0 = Release|Any CPU
 		{48F7884A-9454-4E88-8413-9D35992CB440}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{48F7884A-9454-4E88-8413-9D35992CB440}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{48F7884A-9454-4E88-8413-9D35992CB440}.Release|x86.ActiveCfg = Release|Any CPU
+		{48F7884A-9454-4E88-8413-9D35992CB440}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{48F7884A-9454-4E88-8413-9D35992CB440}.Release35|Any CPU.Build.0 = Release|Any CPU
+		{48F7884A-9454-4E88-8413-9D35992CB440}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{48F7884A-9454-4E88-8413-9D35992CB440}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
+		{48F7884A-9454-4E88-8413-9D35992CB440}.Release35|x86.ActiveCfg = Release|Any CPU
+		{48F7884A-9454-4E88-8413-9D35992CB440}.Release35|x86.Build.0 = Release|Any CPU
 		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Debug|x86.ActiveCfg = Debug|Any CPU
+		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Debug35|Any CPU.Build.0 = Debug|Any CPU
+		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Debug35|Mixed Platforms.Build.0 = Debug|Any CPU
+		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Debug35|x86.ActiveCfg = Debug|Any CPU
+		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Debug35|x86.Build.0 = Debug|Any CPU
 		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Release|Any CPU.Build.0 = Release|Any CPU
 		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Release|x86.ActiveCfg = Release|Any CPU
+		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Release35|Any CPU.Build.0 = Release|Any CPU
+		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
+		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Release35|x86.ActiveCfg = Release|Any CPU
+		{4D77E491-F50F-4A0C-9BD9-F9AB655720AD}.Release35|x86.Build.0 = Release|Any CPU
 		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Debug|x86.ActiveCfg = Debug|Any CPU
+		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Debug35|Any CPU.Build.0 = Debug|Any CPU
+		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Debug35|Mixed Platforms.Build.0 = Debug|Any CPU
+		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Debug35|x86.ActiveCfg = Debug|Any CPU
+		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Debug35|x86.Build.0 = Debug|Any CPU
 		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Release|Any CPU.Build.0 = Release|Any CPU
 		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Release|x86.ActiveCfg = Release|Any CPU
+		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Release35|Any CPU.Build.0 = Release|Any CPU
+		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
+		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Release35|x86.ActiveCfg = Release|Any CPU
+		{E067B8BB-D8E7-4040-BEB8-EFF8BB4149BD}.Release35|x86.Build.0 = Release|Any CPU
 		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Debug|x86.ActiveCfg = Debug|Any CPU
+		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Debug35|Any CPU.Build.0 = Debug|Any CPU
+		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Debug35|Mixed Platforms.Build.0 = Debug|Any CPU
+		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Debug35|x86.ActiveCfg = Debug|Any CPU
+		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Debug35|x86.Build.0 = Debug|Any CPU
 		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Release|Any CPU.Build.0 = Release|Any CPU
 		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Release|x86.ActiveCfg = Release|Any CPU
+		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Release35|Any CPU.Build.0 = Release|Any CPU
+		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
+		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Release35|x86.ActiveCfg = Release|Any CPU
+		{866723F4-E3A4-47C5-A49F-9A68ADD4CFAE}.Release35|x86.Build.0 = Release|Any CPU
 		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Debug|x86.ActiveCfg = Debug|Any CPU
+		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Debug35|Any CPU.Build.0 = Debug|Any CPU
+		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Debug35|Mixed Platforms.Build.0 = Debug|Any CPU
+		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Debug35|x86.ActiveCfg = Debug|Any CPU
+		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Debug35|x86.Build.0 = Debug|Any CPU
 		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Release|Any CPU.Build.0 = Release|Any CPU
 		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Release|x86.ActiveCfg = Release|Any CPU
+		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Release35|Any CPU.Build.0 = Release|Any CPU
+		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
+		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Release35|x86.ActiveCfg = Release|Any CPU
+		{DC83004C-183A-4E1A-ABEA-4FE95B4BC079}.Release35|x86.Build.0 = Release|Any CPU
 		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Debug|x86.ActiveCfg = Debug|Any CPU
+		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Debug35|Any CPU.Build.0 = Debug|Any CPU
+		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Debug35|Mixed Platforms.Build.0 = Debug|Any CPU
+		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Debug35|x86.ActiveCfg = Debug|Any CPU
+		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Debug35|x86.Build.0 = Debug|Any CPU
 		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Release|Any CPU.Build.0 = Release|Any CPU
 		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Release|x86.ActiveCfg = Release|Any CPU
+		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Release35|Any CPU.Build.0 = Release|Any CPU
+		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
+		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Release35|x86.ActiveCfg = Release|Any CPU
+		{F4873D95-4300-4E83-AFFA-EF796495D0F0}.Release35|x86.Build.0 = Release|Any CPU
 		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Debug|x86.ActiveCfg = Debug|Any CPU
+		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Debug35|Any CPU.Build.0 = Debug|Any CPU
+		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Debug35|Mixed Platforms.Build.0 = Debug|Any CPU
+		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Debug35|x86.ActiveCfg = Debug|Any CPU
+		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Debug35|x86.Build.0 = Debug|Any CPU
 		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Release|Any CPU.Build.0 = Release|Any CPU
 		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Release|x86.ActiveCfg = Release|Any CPU
+		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Release35|Any CPU.Build.0 = Release|Any CPU
+		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
+		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Release35|x86.ActiveCfg = Release|Any CPU
+		{D7276EA2-71BE-4208-889B-B1BD48DBE023}.Release35|x86.Build.0 = Release|Any CPU
 		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Debug|x86.ActiveCfg = Debug|Any CPU
+		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Debug35|Any CPU.Build.0 = Debug|Any CPU
+		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Debug35|Mixed Platforms.Build.0 = Debug|Any CPU
+		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Debug35|x86.ActiveCfg = Debug|Any CPU
+		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Debug35|x86.Build.0 = Debug|Any CPU
 		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Release|Any CPU.Build.0 = Release|Any CPU
 		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Release|x86.ActiveCfg = Release|Any CPU
+		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Release35|Any CPU.Build.0 = Release|Any CPU
+		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
+		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Release35|x86.ActiveCfg = Release|Any CPU
+		{3F79B6D4-4359-4F83-B64F-07F4F6262425}.Release35|x86.Build.0 = Release|Any CPU
 		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Debug|x86.ActiveCfg = Debug|Any CPU
 		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Debug|x86.Build.0 = Debug|Any CPU
+		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Debug35|Any CPU.Build.0 = Debug|Any CPU
+		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Debug35|Mixed Platforms.Build.0 = Debug|Any CPU
+		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Debug35|x86.ActiveCfg = Debug|Any CPU
+		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Debug35|x86.Build.0 = Debug|Any CPU
 		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Release|Any CPU.Build.0 = Release|Any CPU
 		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Release|x86.ActiveCfg = Release|Any CPU
 		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Release|x86.Build.0 = Release|Any CPU
+		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Release35|Any CPU.Build.0 = Release|Any CPU
+		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
+		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Release35|x86.ActiveCfg = Release|Any CPU
+		{E8A339C7-FCF6-4A72-8586-56D8961D7B99}.Release35|x86.Build.0 = Release|Any CPU
 		{02BAB603-067D-48B1-AEDD-316849652568}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{02BAB603-067D-48B1-AEDD-316849652568}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{02BAB603-067D-48B1-AEDD-316849652568}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{02BAB603-067D-48B1-AEDD-316849652568}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{02BAB603-067D-48B1-AEDD-316849652568}.Debug|x86.ActiveCfg = Debug|Any CPU
 		{02BAB603-067D-48B1-AEDD-316849652568}.Debug|x86.Build.0 = Debug|Any CPU
+		{02BAB603-067D-48B1-AEDD-316849652568}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{02BAB603-067D-48B1-AEDD-316849652568}.Debug35|Any CPU.Build.0 = Debug|Any CPU
+		{02BAB603-067D-48B1-AEDD-316849652568}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{02BAB603-067D-48B1-AEDD-316849652568}.Debug35|Mixed Platforms.Build.0 = Debug|Any CPU
+		{02BAB603-067D-48B1-AEDD-316849652568}.Debug35|x86.ActiveCfg = Debug|Any CPU
+		{02BAB603-067D-48B1-AEDD-316849652568}.Debug35|x86.Build.0 = Debug|Any CPU
 		{02BAB603-067D-48B1-AEDD-316849652568}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{02BAB603-067D-48B1-AEDD-316849652568}.Release|Any CPU.Build.0 = Release|Any CPU
 		{02BAB603-067D-48B1-AEDD-316849652568}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{02BAB603-067D-48B1-AEDD-316849652568}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{02BAB603-067D-48B1-AEDD-316849652568}.Release|x86.ActiveCfg = Release|Any CPU
 		{02BAB603-067D-48B1-AEDD-316849652568}.Release|x86.Build.0 = Release|Any CPU
+		{02BAB603-067D-48B1-AEDD-316849652568}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{02BAB603-067D-48B1-AEDD-316849652568}.Release35|Any CPU.Build.0 = Release|Any CPU
+		{02BAB603-067D-48B1-AEDD-316849652568}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{02BAB603-067D-48B1-AEDD-316849652568}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
+		{02BAB603-067D-48B1-AEDD-316849652568}.Release35|x86.ActiveCfg = Release|Any CPU
+		{02BAB603-067D-48B1-AEDD-316849652568}.Release35|x86.Build.0 = Release|Any CPU
 		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Debug|x86.ActiveCfg = Debug|Any CPU
 		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Debug|x86.Build.0 = Debug|Any CPU
+		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Debug35|Any CPU.Build.0 = Debug|Any CPU
+		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Debug35|Mixed Platforms.Build.0 = Debug|Any CPU
+		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Debug35|x86.ActiveCfg = Debug|Any CPU
+		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Debug35|x86.Build.0 = Debug|Any CPU
 		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Release|Any CPU.Build.0 = Release|Any CPU
 		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Release|x86.ActiveCfg = Release|Any CPU
 		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Release|x86.Build.0 = Release|Any CPU
+		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Release35|Any CPU.Build.0 = Release|Any CPU
+		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
+		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Release35|x86.ActiveCfg = Release|Any CPU
+		{4C1B794F-8158-45E6-85B3-2C46569BEBC2}.Release35|x86.Build.0 = Release|Any CPU
 		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Debug|x86.ActiveCfg = Debug|Any CPU
+		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Debug35|Any CPU.Build.0 = Debug|Any CPU
+		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Debug35|Mixed Platforms.Build.0 = Debug|Any CPU
+		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Debug35|x86.ActiveCfg = Debug|Any CPU
+		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Debug35|x86.Build.0 = Debug|Any CPU
 		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Release|Any CPU.Build.0 = Release|Any CPU
 		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Release|x86.ActiveCfg = Release|Any CPU
+		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Release35|Any CPU.Build.0 = Release|Any CPU
+		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
+		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Release35|x86.ActiveCfg = Release|Any CPU
+		{4ADD0BBC-B900-4715-9526-D871DE8EEA64}.Release35|x86.Build.0 = Release|Any CPU
 		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Debug|x86.ActiveCfg = Debug|Any CPU
+		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Debug35|Any CPU.Build.0 = Debug|Any CPU
+		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Debug35|Mixed Platforms.Build.0 = Debug|Any CPU
+		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Debug35|x86.ActiveCfg = Debug|Any CPU
+		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Debug35|x86.Build.0 = Debug|Any CPU
 		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Release|Any CPU.Build.0 = Release|Any CPU
 		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Release|x86.ActiveCfg = Release|Any CPU
+		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Release35|Any CPU.Build.0 = Release|Any CPU
+		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
+		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Release35|x86.ActiveCfg = Release|Any CPU
+		{C54FE8FA-7986-4C94-B872-D5BFF7C6C74E}.Release35|x86.Build.0 = Release|Any CPU
 		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Debug|x86.ActiveCfg = Debug|Any CPU
 		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Debug|x86.Build.0 = Debug|Any CPU
+		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Debug35|Any CPU.Build.0 = Debug|Any CPU
+		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Debug35|Mixed Platforms.Build.0 = Debug|Any CPU
+		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Debug35|x86.ActiveCfg = Debug|Any CPU
+		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Debug35|x86.Build.0 = Debug|Any CPU
 		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release|Any CPU.Build.0 = Release|Any CPU
 		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
@@ -397,6 +614,36 @@ Global
 		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Release|x86.ActiveCfg = Release|Any CPU
 		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Release|x86.Build.0 = Release|Any CPU
+		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release35|Any CPU.Build.0 = Release|Any CPU
+		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
+		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release35|x86.ActiveCfg = Release|Any CPU
+		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release35|x86.Build.0 = Release|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Debug|Any CPU.Build.0 = Debug|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Debug|x86.ActiveCfg = Debug|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Debug|x86.Build.0 = Debug|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Debug35|Any CPU.Build.0 = Debug|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Debug35|Mixed Platforms.Build.0 = Debug|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Debug35|x86.ActiveCfg = Debug|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Debug35|x86.Build.0 = Debug|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Release|Any CPU.ActiveCfg = Release|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Release|Any CPU.Build.0 = Release|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Release|Mixed Platforms.Build.0 = Release|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Release|x86.ActiveCfg = Release|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Release|x86.Build.0 = Release|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Release35|Any CPU.Build.0 = Release|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Release35|x86.ActiveCfg = Release|Any CPU
+		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Release35|x86.Build.0 = Release|Any CPU
 	EndGlobalSection
 	GlobalSection(SolutionProperties) = preSolution
 		HideSolutionNode = FALSE

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Core/Search/BitsFilteredDocIdSet.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Core/Search/BitsFilteredDocIdSet.cs b/src/Lucene.Net.Core/Search/BitsFilteredDocIdSet.cs
index ac3598d..ec39eb5 100644
--- a/src/Lucene.Net.Core/Search/BitsFilteredDocIdSet.cs
+++ b/src/Lucene.Net.Core/Search/BitsFilteredDocIdSet.cs
@@ -56,7 +56,7 @@ namespace Lucene.Net.Search
             this.AcceptDocs = acceptDocs;
         }
 
-        protected internal override bool Match(int docid)
+        protected override bool Match(int docid)
         {
             return AcceptDocs.Get(docid);
         }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Core/Search/FilteredDocIdSet.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Core/Search/FilteredDocIdSet.cs b/src/Lucene.Net.Core/Search/FilteredDocIdSet.cs
index 7034971..1808617 100644
--- a/src/Lucene.Net.Core/Search/FilteredDocIdSet.cs
+++ b/src/Lucene.Net.Core/Search/FilteredDocIdSet.cs
@@ -92,7 +92,7 @@ namespace Lucene.Net.Search
         /// Validation method to determine whether a docid should be in the result set. </summary>
         /// <param name="docid"> docid to be tested </param>
         /// <returns> true if input docid should be in the result set, false otherwise. </returns>
-        protected internal abstract bool Match(int docid);
+        protected abstract bool Match(int docid);
 
         /// <summary>
         /// Implementation of the contract to build a DocIdSetIterator. </summary>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Contrib.Spatial.csproj
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Contrib.Spatial.csproj b/src/Lucene.Net.Spatial/Contrib.Spatial.csproj
deleted file mode 100644
index ff13d0c..0000000
--- a/src/Lucene.Net.Spatial/Contrib.Spatial.csproj
+++ /dev/null
@@ -1,202 +0,0 @@
-\ufeff<?xml version="1.0" encoding="utf-8"?>
-<!--
-
- 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.
-
--->
-<Project ToolsVersion="4.0" DefaultTargets="Build" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
-  <PropertyGroup>
-    <Configuration Condition=" '$(Configuration)' == '' ">Debug</Configuration>
-    <Platform Condition=" '$(Platform)' == '' ">AnyCPU</Platform>
-    <ProductVersion>9.0.21022</ProductVersion>
-    <SchemaVersion>2.0</SchemaVersion>
-    <ProjectGuid>{35C347F4-24B2-4BE5-8117-A0E3001551CE}</ProjectGuid>
-    <AppDesignerFolder>Properties</AppDesignerFolder>
-    <RootNamespace>Lucene.Net.Spatial</RootNamespace>
-    <AssemblyName>Lucene.Net.Contrib.Spatial</AssemblyName>
-    <FileAlignment>512</FileAlignment>
-    <FileUpgradeFlags>
-    </FileUpgradeFlags>
-    <OldToolsVersion>3.5</OldToolsVersion>
-    <UpgradeBackupLocation />
-    <PublishUrl>publish\</PublishUrl>
-    <Install>true</Install>
-    <InstallFrom>Disk</InstallFrom>
-    <UpdateEnabled>false</UpdateEnabled>
-    <UpdateMode>Foreground</UpdateMode>
-    <UpdateInterval>7</UpdateInterval>
-    <UpdateIntervalUnits>Days</UpdateIntervalUnits>
-    <UpdatePeriodically>false</UpdatePeriodically>
-    <UpdateRequired>false</UpdateRequired>
-    <MapFileExtensions>true</MapFileExtensions>
-    <ApplicationRevision>0</ApplicationRevision>
-    <ApplicationVersion>1.0.0.%2a</ApplicationVersion>
-    <IsWebBootstrapper>false</IsWebBootstrapper>
-    <UseApplicationTrust>false</UseApplicationTrust>
-    <BootstrapperEnabled>true</BootstrapperEnabled>
-  </PropertyGroup>
-  <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Debug|AnyCPU' ">
-    <TargetFrameworkVersion>v4.0</TargetFrameworkVersion>
-    <Framework>$(TargetFrameworkVersion.Replace("v", "NET").Replace(".", ""))</Framework>
-    <DebugSymbols>true</DebugSymbols>
-    <DebugType>full</DebugType>
-    <Optimize>false</Optimize>
-    <OutputPath>..\..\..\build\bin\contrib\Spatial\$(Configuration.Replace("35", ""))\$(Framework)\</OutputPath>
-    <DefineConstants>DEBUG;TRACE;$(Framework)</DefineConstants>
-    <ErrorReport>prompt</ErrorReport>
-    <WarningLevel>4</WarningLevel>
-    <DocumentationFile>
-    </DocumentationFile>
-    <NoWarn>618</NoWarn>
-    <OutputType>Library</OutputType>
-  </PropertyGroup>
-  <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Debug35|AnyCPU' ">
-    <TargetFrameworkVersion>v3.5</TargetFrameworkVersion>
-    <Framework>$(TargetFrameworkVersion.Replace("v", "NET").Replace(".", ""))</Framework>
-    <DebugSymbols>true</DebugSymbols>
-    <DebugType>full</DebugType>
-    <Optimize>false</Optimize>
-    <OutputPath>..\..\..\build\bin\contrib\Spatial\$(Configuration.Replace("35", ""))\$(Framework)\</OutputPath>
-    <DefineConstants>DEBUG;TRACE;$(Framework)</DefineConstants>
-    <ErrorReport>prompt</ErrorReport>
-    <WarningLevel>4</WarningLevel>
-    <DocumentationFile>
-    </DocumentationFile>
-    <NoWarn>618</NoWarn>
-    <OutputType>Library</OutputType>
-  </PropertyGroup>
-  <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Release|AnyCPU' ">
-    <TargetFrameworkVersion>v4.0</TargetFrameworkVersion>
-    <Framework>$(TargetFrameworkVersion.Replace("v", "NET").Replace(".", ""))</Framework>
-    <DebugType>pdbonly</DebugType>
-    <Optimize>true</Optimize>
-    <OutputPath>..\..\..\build\bin\contrib\Spatial\$(Configuration.Replace("35", ""))\$(Framework)\</OutputPath>
-    <DefineConstants>TRACE;$(Framework)</DefineConstants>
-    <ErrorReport>prompt</ErrorReport>
-    <WarningLevel>4</WarningLevel>
-    <DocumentationFile>..\..\..\build\bin\contrib\Spatial\$(Configuration.Replace("35", ""))\$(Framework)\Lucene.Net.Contrib.Spatial.XML</DocumentationFile>
-    <NoWarn>618</NoWarn>
-    <DebugSymbols>true</DebugSymbols>
-    <OutputType>Library</OutputType>
-  </PropertyGroup>
-  <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Release35|AnyCPU' ">
-    <TargetFrameworkVersion>v3.5</TargetFrameworkVersion>
-    <Framework>$(TargetFrameworkVersion.Replace("v", "NET").Replace(".", ""))</Framework>
-    <DebugType>pdbonly</DebugType>
-    <Optimize>true</Optimize>
-    <OutputPath>..\..\..\build\bin\contrib\Spatial\$(Configuration.Replace("35", ""))\$(Framework)\</OutputPath>
-    <DefineConstants>TRACE;$(Framework)</DefineConstants>
-    <ErrorReport>prompt</ErrorReport>
-    <WarningLevel>4</WarningLevel>
-    <DocumentationFile>..\..\..\build\bin\contrib\Spatial\$(Configuration.Replace("35", ""))\$(Framework)\Lucene.Net.Contrib.Spatial.XML</DocumentationFile>
-    <NoWarn>618</NoWarn>
-    <DebugSymbols>true</DebugSymbols>
-    <OutputType>Library</OutputType>
-  </PropertyGroup>
-  <PropertyGroup>
-    <SignAssembly>true</SignAssembly>
-  </PropertyGroup>
-  <PropertyGroup>
-    <AssemblyOriginatorKeyFile>Lucene.Net.snk</AssemblyOriginatorKeyFile>
-  </PropertyGroup>
-  <ItemGroup>
-    <Reference Include="Spatial4n.Core">
-      <HintPath>..\..\..\lib\Spatial4n\$(Framework)\Spatial4n.Core.dll</HintPath>
-    </Reference>
-    <Reference Include="System" />
-    <Reference Condition="'$(Framework)' == 'NET35'" Include="System.Core" />
-    <Reference Include="System.Data" />
-  </ItemGroup>
-  <ItemGroup>
-    <Compile Include="BBox\AreaSimilarity.cs" />
-    <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" />
-    <Compile Include="Prefix\RecursivePrefixTreeStrategy.cs" />
-    <Compile Include="Prefix\TermQueryPrefixTreeStrategy.cs" />
-    <Compile Include="Prefix\Tree\GeohashPrefixTree.cs" />
-    <Compile Include="Prefix\Tree\Node.cs" />
-    <Compile Include="Prefix\Tree\QuadPrefixTree.cs" />
-    <Compile Include="Prefix\Tree\SpatialPrefixTree.cs" />
-    <Compile Include="Prefix\Tree\SpatialPrefixTreeFactory.cs" />
-    <Compile Include="Properties\AssemblyInfo.cs" />
-    <Compile Include="Queries\SpatialArgs.cs" />
-    <Compile Include="Queries\SpatialArgsParser.cs" />
-    <Compile Include="Queries\SpatialOperation.cs" />
-    <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" />
-    <Compile Include="Util\FixedBitSet.cs" />
-    <Compile Include="Util\FunctionQuery.cs" />
-    <Compile Include="Util\ShapeFieldCache.cs" />
-    <Compile Include="Util\ShapeFieldCacheProvider.cs" />
-    <Compile Include="Util\TermsEnumCompatibility.cs" />
-    <Compile Include="Util\TermsFilter.cs" />
-    <Compile Include="Util\ValueSourceFilter.cs" />
-    <Compile Include="Vector\DistanceValueSource.cs" />
-    <Compile Include="Vector\PointVectorStrategy.cs" />
-  </ItemGroup>
-  <ItemGroup>
-    <ProjectReference Include="..\..\core\Lucene.Net.csproj">
-      <Project>{5D4AD9BE-1FFB-41AB-9943-25737971BF57}</Project>
-      <Name>Lucene.Net</Name>
-    </ProjectReference>
-  </ItemGroup>
-  <ItemGroup>
-    <BootstrapperPackage Include=".NETFramework,Version=v4.0">
-      <Visible>False</Visible>
-      <ProductName>Microsoft .NET Framework 4 %28x86 and x64%29</ProductName>
-      <Install>true</Install>
-    </BootstrapperPackage>
-    <BootstrapperPackage Include="Microsoft.Net.Client.3.5">
-      <Visible>False</Visible>
-      <ProductName>.NET Framework 3.5 SP1 Client Profile</ProductName>
-      <Install>false</Install>
-    </BootstrapperPackage>
-    <BootstrapperPackage Include="Microsoft.Net.Framework.3.5.SP1">
-      <Visible>False</Visible>
-      <ProductName>.NET Framework 3.5 SP1</ProductName>
-      <Install>false</Install>
-    </BootstrapperPackage>
-    <BootstrapperPackage Include="Microsoft.Windows.Installer.3.1">
-      <Visible>False</Visible>
-      <ProductName>Windows Installer 3.1</ProductName>
-      <Install>true</Install>
-    </BootstrapperPackage>
-  </ItemGroup>
-  <ItemGroup>
-    <None Include="Lucene.Net.snk" />
-  </ItemGroup>
-  <ItemGroup />
-  <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
-  <!-- To modify your build process, add your task inside one of the targets below and uncomment it. 
-       Other similar extension points exist, see Microsoft.Common.targets.
-  <Target Name="BeforeBuild">
-  </Target>
-  <Target Name="AfterBuild">
-  </Target>
-  -->
-</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs b/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
index ab6fa3f..e36a97b 100644
--- a/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
+++ b/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
@@ -16,6 +16,7 @@
  */
 using System;
 using Lucene.Net.Index;
+using Lucene.Net.Queries;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Queries;
 using Lucene.Net.Util;
@@ -40,8 +41,7 @@ namespace Lucene.Net.Spatial
     /// <lucene.experimental></lucene.experimental>
     public class DisjointSpatialFilter : Filter
     {
-        private readonly string field;
-
+        private readonly string field;//maybe null
         private readonly Filter intersectsFilter;
 
         /// <param name="strategy">Needed to compute intersects</param>
@@ -54,16 +54,12 @@ namespace Lucene.Net.Spatial
         /// .
         /// Passing null will assume all docs have spatial data.
         /// </param>
-        public DisjointSpatialFilter(SpatialStrategy strategy, SpatialArgs args, string field
-            )
+        public DisjointSpatialFilter(SpatialStrategy strategy, SpatialArgs args, string field)
         {
-            //maybe null
             this.field = field;
             // TODO consider making SpatialArgs cloneable
-            SpatialOperation origOp = args.Operation;
-            //copy so we can restore
-            args.Operation = SpatialOperation.Intersects;
-            //temporarily set to intersects
+            SpatialOperation origOp = args.Operation; //copy so we can restore
+            args.Operation = SpatialOperation.Intersects; //temporarily set to intersects
             intersectsFilter = strategy.MakeFilter(args);
             args.Operation = origOp;
         }
@@ -99,45 +95,37 @@ namespace Lucene.Net.Spatial
         }
 
         /// <exception cref="System.IO.IOException"></exception>
-        public override DocIdSet GetDocIdSet(AtomicReaderContext context, IBits acceptDocs
-            )
+        public override DocIdSet GetDocIdSet(AtomicReaderContext context, Bits acceptDocs)
         {
-            IBits docsWithField;
+            Bits docsWithField;
             if (field == null)
             {
                 docsWithField = null;
             }
             else
             {
-                //all docs
                 //NOTE By using the FieldCache we re-use a cache
                 // which is nice but loading it in this way might be slower than say using an
                 // intersects filter against the world bounds. So do we add a method to the
                 // strategy, perhaps?  But the strategy can't cache it.
                 docsWithField = FieldCache.DEFAULT.GetDocsWithField((context.AtomicReader), field);
                 int maxDoc = context.AtomicReader.MaxDoc;
-                if (docsWithField.Length != maxDoc)
+                if (docsWithField.Length() != maxDoc)
                 {
-                    throw new InvalidOperationException("Bits length should be maxDoc (" + maxDoc + ") but wasn't: "
-                                                        + docsWithField);
+                    throw new InvalidOperationException("Bits length should be maxDoc (" + maxDoc + ") but wasn't: " + docsWithField);
                 }
-                if (docsWithField is Bits.MatchNoBits)
+                if (docsWithField is Bits_MatchNoBits)
                 {
-                    return null;
+                    return null;//match nothing
                 }
-                else
+                else if (docsWithField is Bits_MatchAllBits)
                 {
-                    //match nothing
-                    if (docsWithField is Bits.MatchAllBits)
-                    {
-                        docsWithField = null;
-                    }
+                    docsWithField = null;//all docs
                 }
             }
-            //all docs
+            
             //not so much a chain but a way to conveniently invert the Filter
-            DocIdSet docIdSet = new ChainedFilter(new[] { intersectsFilter }, ChainedFilter.ANDNOT).GetDocIdSet(context,
-                                                                                                              acceptDocs);
+            DocIdSet docIdSet = new ChainedFilter(new[] { intersectsFilter }, ChainedFilter.ANDNOT).GetDocIdSet(context, acceptDocs);
             return BitsFilteredDocIdSet.Wrap(docIdSet, docsWithField);
         }
     }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj b/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj
new file mode 100644
index 0000000..fabf96c
--- /dev/null
+++ b/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj
@@ -0,0 +1,119 @@
+\ufeff<?xml version="1.0" encoding="utf-8"?>
+<!--
+
+ 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.
+
+-->
+<Project ToolsVersion="12.0" DefaultTargets="Build" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
+  <PropertyGroup>
+    <Configuration Condition=" '$(Configuration)' == '' ">Debug</Configuration>
+    <Platform Condition=" '$(Platform)' == '' ">AnyCPU</Platform>
+    <ProjectGuid>{35C347F4-24B2-4BE5-8117-A0E3001551CE}</ProjectGuid>
+    <AppDesignerFolder>Properties</AppDesignerFolder>
+    <RootNamespace>Lucene.Net.Spatial</RootNamespace>
+    <AssemblyName>Lucene.Net.Spatial</AssemblyName>
+    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <FileAlignment>512</FileAlignment>
+    <TargetFrameworkProfile />
+  </PropertyGroup>
+  <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Debug|AnyCPU' ">
+    <DebugSymbols>true</DebugSymbols>
+    <DebugType>full</DebugType>
+    <Optimize>false</Optimize>
+    <OutputPath>bin\Debug\</OutputPath>
+    <DefineConstants>DEBUG;TRACE</DefineConstants>
+    <ErrorReport>prompt</ErrorReport>
+    <WarningLevel>4</WarningLevel>
+    <Prefer32Bit>false</Prefer32Bit>
+  </PropertyGroup>
+  <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Release|AnyCPU' ">
+    <DebugType>pdbonly</DebugType>
+    <Optimize>true</Optimize>
+    <OutputPath>bin\Release\</OutputPath>
+    <DefineConstants>TRACE</DefineConstants>
+    <ErrorReport>prompt</ErrorReport>
+    <WarningLevel>4</WarningLevel>
+    <Prefer32Bit>false</Prefer32Bit>
+  </PropertyGroup>
+  <PropertyGroup>
+    <OutputType>Library</OutputType>
+  </PropertyGroup>
+  <PropertyGroup>
+    <StartupObject />
+  </PropertyGroup>
+  <ItemGroup>
+    <Reference Include="Spatial4n.Core, Version=0.3.0.0, Culture=neutral, PublicKeyToken=9f9456e1ca16d45e, processorArchitecture=MSIL">
+      <HintPath>..\..\packages\Spatial4n.Core.0.3\lib\net40\Spatial4n.Core.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="System" />
+    <Reference Include="System.Core" />
+    <Reference Include="Microsoft.CSharp" />
+  </ItemGroup>
+  <ItemGroup>
+    <Compile Include="DisjointSpatialFilter.cs" />
+    <Compile Include="Prefix\AbstractPrefixTreeFilter.cs" />
+    <Compile Include="Prefix\AbstractVisitingPrefixTreeFilter.cs" />
+    <Compile Include="Prefix\ContainsPrefixTreeFilter.cs" />
+    <Compile Include="Prefix\IntersectsPrefixTreeFilter.cs" />
+    <Compile Include="Prefix\PointPrefixTreeFieldCacheProvider.cs" />
+    <Compile Include="Prefix\PrefixTreeStrategy.cs" />
+    <Compile Include="Prefix\RecursivePrefixTreeStrategy.cs" />
+    <Compile Include="Prefix\TermQueryPrefixTreeStrategy.cs" />
+    <Compile Include="Prefix\Tree\Cell.cs" />
+    <Compile Include="Prefix\Tree\GeohashPrefixTree.cs" />
+    <Compile Include="Prefix\Tree\QuadPrefixTree.cs" />
+    <Compile Include="Prefix\Tree\SpatialPrefixTree.cs" />
+    <Compile Include="Prefix\Tree\SpatialPrefixTreeFactory.cs" />
+    <Compile Include="Prefix\WithinPrefixTreeFilter.cs" />
+    <Compile Include="Properties\AssemblyInfo.cs" />
+    <Compile Include="Queries\SpatialArgs.cs" />
+    <Compile Include="Queries\SpatialArgsParser.cs" />
+    <Compile Include="Queries\SpatialOperation.cs" />
+    <Compile Include="Queries\UnsupportedSpatialOperation.cs" />
+    <Compile Include="SpatialStrategy.cs" />
+    <Compile Include="Util\ShapeFieldCacheDistanceValueSource.cs" />
+    <Compile Include="Util\CachingDoubleValueSource.cs" />
+    <Compile Include="Util\ShapeFieldCache.cs" />
+    <Compile Include="Util\ShapeFieldCacheProvider.cs" />
+    <Compile Include="Util\ValueSourceFilter.cs" />
+    <Compile Include="Vector\DistanceValueSource.cs" />
+    <Compile Include="Vector\PointVectorStrategy.cs" />
+  </ItemGroup>
+  <ItemGroup>
+    <ProjectReference Include="..\Lucene.Net.Core\Lucene.Net.csproj">
+      <Project>{5d4ad9be-1ffb-41ab-9943-25737971bf57}</Project>
+      <Name>Lucene.Net</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\Lucene.Net.Queries\Lucene.Net.Queries.csproj">
+      <Project>{69D7956C-C2CC-4708-B399-A188FEC384C4}</Project>
+      <Name>Lucene.Net.Queries</Name>
+    </ProjectReference>
+  </ItemGroup>
+  <ItemGroup>
+    <None Include="packages.config" />
+  </ItemGroup>
+  <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
+  <!-- To modify your build process, add your task inside one of the targets below and uncomment it. 
+       Other similar extension points exist, see Microsoft.Common.targets.
+  <Target Name="BeforeBuild">
+  </Target>
+  <Target Name="AfterBuild">
+  </Target>
+  -->
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
index efcf078..c70a7c5 100644
--- a/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
@@ -92,14 +92,14 @@ namespace Lucene.Net.Spatial.Prefix
             protected internal readonly AtomicReaderContext context;
 
             protected internal readonly int maxDoc;
-            protected internal IBits acceptDocs;
+            protected internal Bits acceptDocs;
 
             protected internal DocsEnum docsEnum;
             protected internal TermsEnum termsEnum;
 
             /// <exception cref="System.IO.IOException"></exception>
             public BaseTermsEnumTraverser(AbstractPrefixTreeFilter _enclosing, AtomicReaderContext
-                                                                                   context, IBits acceptDocs)
+                                                                                   context, Bits acceptDocs)
             {
                 this._enclosing = _enclosing;
                 //remember to check for null in getDocIdSet

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
index f187216..980f4cc 100644
--- a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
@@ -180,7 +180,7 @@ namespace Lucene.Net.Spatial.Prefix
 
             /// <exception cref="System.IO.IOException"></exception>
             public VisitorTemplate(AbstractVisitingPrefixTreeFilter _enclosing, AtomicReaderContext
-                                                                                    context, IBits acceptDocs,
+                                                                                    context, Bits acceptDocs,
                                    bool hasIndexedLeaves)
                 : base(_enclosing, context, acceptDocs)
             {
@@ -250,31 +250,27 @@ namespace Lucene.Net.Spatial.Prefix
                         }
                     }
                     //Seek to curVNode's cell (or skip if termsEnum has moved beyond)
-                    curVNodeTerm.bytes = curVNode.cell.GetTokenBytes().ToSByteArray();
-                    curVNodeTerm.length = curVNodeTerm.bytes.Length;
+                    curVNodeTerm.Bytes = curVNode.cell.GetTokenBytes();
+                    curVNodeTerm.Length = curVNodeTerm.Bytes.Length;
                     int compare = termsEnum.Comparator.Compare(thisTerm, curVNodeTerm
                         );
                     if (compare > 0)
                     {
                         // leap frog (termsEnum is beyond where we would otherwise seek)
-                        Debug.Assert(
-                            !((AtomicReader)context.Reader).Terms(_enclosing.fieldName).Iterator(null).SeekExact(
-                                curVNodeTerm, false), "should be absent"
-                            );
+                        Debug.Assert(!((AtomicReader)context.Reader).Terms(_enclosing.fieldName).Iterator(null).SeekExact(curVNodeTerm), "should be absent");
                     }
                     else
                     {
                         if (compare < 0)
                         {
                             // Seek !
-                            TermsEnum.SeekStatus seekStatus = termsEnum.SeekCeil(curVNodeTerm, true
-                                );
+                            TermsEnum.SeekStatus seekStatus = termsEnum.SeekCeil(curVNodeTerm);
                             if (seekStatus == TermsEnum.SeekStatus.END)
                             {
                                 break;
                             }
                             // all done
-                            thisTerm = termsEnum.Term;
+                            thisTerm = termsEnum.Term();
                             if (seekStatus == TermsEnum.SeekStatus.NOT_FOUND)
                             {
                                 continue;
@@ -324,8 +320,8 @@ namespace Lucene.Net.Spatial.Prefix
                     // then add all of those docs
                     Debug.Assert(StringHelper.StartsWith(thisTerm, curVNodeTerm
                                      ));
-                    scanCell = _enclosing.grid.GetCell(thisTerm.bytes.ToByteArray(), thisTerm.offset
-                                                       , thisTerm.length, scanCell);
+                    scanCell = _enclosing.grid.GetCell(thisTerm.Bytes, thisTerm.Offset
+                                                       , thisTerm.Length, scanCell);
                     if (scanCell.Level == cell.Level && scanCell.IsLeaf())
                     {
                         VisitLeaf(scanCell);
@@ -397,8 +393,8 @@ namespace Lucene.Net.Spatial.Prefix
                                             );
                     thisTerm = termsEnum.Next())
                 {
-                    scanCell = _enclosing.grid.GetCell(thisTerm.bytes.ToByteArray(), thisTerm.offset
-                                                       , thisTerm.length, scanCell);
+                    scanCell = _enclosing.grid.GetCell(thisTerm.Bytes, thisTerm.Offset
+                                                       , thisTerm.Length, scanCell);
                     int termLevel = scanCell.Level;
                     if (termLevel > scanDetailLevel)
                     {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
index bd13b79..23fb602 100644
--- a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
@@ -37,39 +37,45 @@ namespace Lucene.Net.Spatial.Prefix
     /// <lucene.experimental></lucene.experimental>
     public class ContainsPrefixTreeFilter : AbstractPrefixTreeFilter
     {
-        public ContainsPrefixTreeFilter(Shape queryShape, string
-             fieldName, SpatialPrefixTree grid, int detailLevel)
+        protected readonly bool multiOverlappingIndexedShapes;
+
+        public ContainsPrefixTreeFilter(Shape queryShape, string fieldName, SpatialPrefixTree grid, int detailLevel, bool multiOverlappingIndexedShapes)
             : base(queryShape, fieldName, grid, detailLevel)
         {
+            this.multiOverlappingIndexedShapes = multiOverlappingIndexedShapes;
         }
 
         /// <exception cref="System.IO.IOException"></exception>
-        public override DocIdSet GetDocIdSet(AtomicReaderContext context, IBits acceptDocs
-            )
+        public override DocIdSet GetDocIdSet(AtomicReaderContext context, Bits acceptDocs)
         {
             return new ContainsVisitor(this, context, acceptDocs).Visit(grid.WorldCell, acceptDocs);
         }
 
         private class ContainsVisitor : BaseTermsEnumTraverser
         {
+            private readonly Shape queryShape;
+            private readonly int detailLevel;
+            private readonly bool multiOverlappingIndexedShapes;
+            private SpatialPrefixTree grid;
+
             /// <exception cref="System.IO.IOException"></exception>
-            public ContainsVisitor(ContainsPrefixTreeFilter _enclosing, AtomicReaderContext context
-                , IBits acceptDocs)
-                : base(_enclosing, context, acceptDocs)
+            public ContainsVisitor(ContainsPrefixTreeFilter enclosing, AtomicReaderContext context
+                , Bits acceptDocs)
+                : base(enclosing, context, acceptDocs)
             {
-                this._enclosing = _enclosing;
+                this.queryShape = enclosing.queryShape;
+                this.detailLevel = enclosing.detailLevel;
+                this.grid = enclosing.grid;
+                this.multiOverlappingIndexedShapes = enclosing.multiOverlappingIndexedShapes;
             }
 
             internal BytesRef termBytes = new BytesRef();
 
-            internal Cell nextCell;
+            internal Cell nextCell;//see getLeafDocs
 
-            //see getLeafDocs
-            /// <summary>This is the primary algorithm; recursive.</summary>
             /// <remarks>This is the primary algorithm; recursive.  Returns null if finds none.</remarks>
             /// <exception cref="System.IO.IOException"></exception>
-            internal SmallDocSet Visit(Cell cell, IBits acceptContains
-                )
+            internal SmallDocSet Visit(Cell cell, Bits acceptContains)
             {
                 if (termsEnum == null)
                 {
@@ -80,39 +86,43 @@ namespace Lucene.Net.Spatial.Prefix
                 SmallDocSet leafDocs = GetLeafDocs(cell, acceptContains);
                 // Get the AND of all child results
                 SmallDocSet combinedSubResults = null;
-                ICollection<Cell> subCells = cell.GetSubCells(_enclosing.queryShape);
+
+                //   Optimization: use null subCellsFilter when we know cell is within the query shape.
+                Shape subCellsFilter = queryShape;
+                if (cell.Level != 0 && ((cell.GetShapeRel() == null || cell.GetShapeRel() == SpatialRelation.WITHIN)))
+                {
+                    subCellsFilter = null;
+                    System.Diagnostics.Debug.Assert(cell.GetShape().Relate(queryShape) == SpatialRelation.WITHIN);
+                }
+                ICollection<Cell> subCells = cell.GetSubCells(subCellsFilter);
                 foreach (Cell subCell in subCells)
                 {
                     if (!SeekExact(subCell))
                     {
                         combinedSubResults = null;
                     }
+                    else if (subCell.Level == detailLevel)
+                    {
+                        combinedSubResults = GetDocs(subCell, acceptContains);
+                    }
+                    else if (!multiOverlappingIndexedShapes && 
+                        subCell.GetShapeRel() == SpatialRelation.WITHIN)
+                    {
+                        combinedSubResults = GetLeafDocs(subCell, acceptContains);
+                    }
                     else
                     {
-                        if (subCell.Level == _enclosing.detailLevel)
-                        {
-                            combinedSubResults = GetDocs(subCell, acceptContains);
-                        }
-                        else
-                        {
-                            if (subCell.GetShapeRel() == SpatialRelation.WITHIN)
-                            {
-                                combinedSubResults = GetLeafDocs(subCell, acceptContains);
-                            }
-                            else
-                            {
-                                combinedSubResults = Visit(subCell, acceptContains);
-                            }
-                        }
+                        combinedSubResults = Visit(subCell, acceptContains);
                     }
-                    //recursion
+                    
                     if (combinedSubResults == null)
                     {
                         break;
                     }
-                    acceptContains = combinedSubResults;
+
+                    acceptContains = combinedSubResults;//has the 'AND' effect on next iteration
                 }
-                //has the 'AND' effect on next iteration
+                
                 // Result: OR the leaf docs with AND of all child results
                 if (combinedSubResults != null)
                 {
@@ -128,39 +138,43 @@ namespace Lucene.Net.Spatial.Prefix
             /// <exception cref="System.IO.IOException"></exception>
             private bool SeekExact(Cell cell)
             {
-                System.Diagnostics.Debug.Assert(new BytesRef(cell.GetTokenBytes().ToSByteArray()).CompareTo(this
-                    .termBytes) > 0);
-                this.termBytes.bytes = cell.GetTokenBytes().ToSByteArray();
-                this.termBytes.length = this.termBytes.bytes.Length;
-                return this.termsEnum.SeekExact(this.termBytes, cell.Level <= 2);
+                System.Diagnostics.Debug.Assert(new BytesRef(cell.GetTokenBytes()).CompareTo(termBytes) > 0);
+                this.termBytes.Bytes = cell.GetTokenBytes();
+                this.termBytes.Length = this.termBytes.Bytes.Length;
+                if (termsEnum == null)
+                    return false;
+                return this.termsEnum.SeekExact(termBytes);
             }
 
             /// <exception cref="System.IO.IOException"></exception>
-            private ContainsPrefixTreeFilter.SmallDocSet GetDocs(Cell cell, IBits acceptContains
-                )
+            private SmallDocSet GetDocs(Cell cell, Bits acceptContains)
             {
-                System.Diagnostics.Debug.Assert(new BytesRef(cell.GetTokenBytes().ToSByteArray()).Equals(this.termBytes
-                    ));
+                System.Diagnostics.Debug.Assert(new BytesRef(cell.GetTokenBytes()).Equals(termBytes));
                 return this.CollectDocs(acceptContains);
             }
 
+            private Cell lastLeaf = null;//just for assertion
+
             /// <exception cref="System.IO.IOException"></exception>
-            private ContainsPrefixTreeFilter.SmallDocSet GetLeafDocs(Cell leafCell, IBits acceptContains)
+            private SmallDocSet GetLeafDocs(Cell leafCell, Bits acceptContains)
             {
-                System.Diagnostics.Debug.Assert(new BytesRef(leafCell.GetTokenBytes().ToSByteArray()).Equals(this
-                    .termBytes));
+                System.Diagnostics.Debug.Assert(new BytesRef(leafCell.GetTokenBytes()).Equals(termBytes));
+                System.Diagnostics.Debug.Assert(leafCell.Equals(lastLeaf));//don't call for same leaf again
+                lastLeaf = leafCell;
+
+                if (termsEnum == null)
+                    return null;
                 BytesRef nextTerm = this.termsEnum.Next();
                 if (nextTerm == null)
                 {
-                    this.termsEnum = null;
+                    termsEnum = null;
                     //signals all done
                     return null;
                 }
-                this.nextCell = this._enclosing.grid.GetCell(nextTerm.bytes.ToByteArray(), nextTerm.offset, nextTerm
-                    .length, this.nextCell);
-                if (this.nextCell.Level == leafCell.Level && this.nextCell.IsLeaf())
+                nextCell = grid.GetCell(nextTerm.Bytes, nextTerm.Offset, nextTerm.Length, this.nextCell);
+                if (nextCell.Level == leafCell.Level && nextCell.IsLeaf())
                 {
-                    return this.CollectDocs(acceptContains);
+                    return CollectDocs(acceptContains);
                 }
                 else
                 {
@@ -169,38 +183,34 @@ namespace Lucene.Net.Spatial.Prefix
             }
 
             /// <exception cref="System.IO.IOException"></exception>
-            private ContainsPrefixTreeFilter.SmallDocSet CollectDocs(IBits acceptContains)
+            private SmallDocSet CollectDocs(Bits acceptContains)
             {
-                ContainsPrefixTreeFilter.SmallDocSet set = null;
-                this.docsEnum = this.termsEnum.Docs(acceptContains, this.docsEnum, DocsEnum.FLAG_NONE
-                    );
+                SmallDocSet set = null;
+                docsEnum = termsEnum.Docs(acceptContains, docsEnum, DocsEnum.FLAG_NONE);
                 int docid;
-                while ((docid = this.docsEnum.NextDoc()) != DocIdSetIterator.NO_MORE_DOCS)
+                while ((docid = docsEnum.NextDoc()) != DocIdSetIterator.NO_MORE_DOCS)
                 {
                     if (set == null)
                     {
-                        int size = this.termsEnum.DocFreq;
+                        int size = this.termsEnum.DocFreq();
                         if (size <= 0)
                         {
                             size = 16;
                         }
-                        set = new ContainsPrefixTreeFilter.SmallDocSet(size);
+                        set = new SmallDocSet(size);
                     }
                     set.Set(docid);
                 }
                 return set;
             }
-
-            private readonly ContainsPrefixTreeFilter _enclosing;
-            //class ContainsVisitor
-        }
+        }//class ContainsVisitor
 
         /// <summary>A hash based mutable set of docIds.</summary>
         /// <remarks>
         /// A hash based mutable set of docIds. If this were Solr code then we might
         /// use a combination of HashDocSet and SortedIntDocSet instead.
         /// </remarks>
-        private class SmallDocSet : DocIdSet, Lucene.Net.Util.IBits
+        private class SmallDocSet : DocIdSet, Bits
         {
             private readonly SentinelIntSet intSet;
 
@@ -234,7 +244,7 @@ namespace Lucene.Net.Spatial.Prefix
             /// <remarks>Number of docids.</remarks>
             public virtual int Size()
             {
-                return intSet.Size;
+                return intSet.Size();
             }
 
             /// <summary>NOTE: modifies and returns either "this" or "other"</summary>
@@ -243,7 +253,7 @@ namespace Lucene.Net.Spatial.Prefix
             {
                 ContainsPrefixTreeFilter.SmallDocSet bigger;
                 ContainsPrefixTreeFilter.SmallDocSet smaller;
-                if (other.intSet.Size > this.intSet.Size)
+                if (other.intSet.Size() > this.intSet.Size())
                 {
                     bigger = other;
                     smaller = this;
@@ -254,9 +264,9 @@ namespace Lucene.Net.Spatial.Prefix
                     smaller = other;
                 }
                 //modify bigger
-                foreach (int v in smaller.intSet.keys)
+                foreach (int v in smaller.intSet.Keys)
                 {
-                    if (v == smaller.intSet.emptyVal)
+                    if (v == smaller.intSet.EmptyVal)
                     {
                         continue;
                     }
@@ -266,7 +276,7 @@ namespace Lucene.Net.Spatial.Prefix
             }
 
             /// <exception cref="System.IO.IOException"></exception>
-            public override Lucene.Net.Util.IBits Bits
+            public Lucene.Net.Util.Bits Bits
             {
                 get
                 {
@@ -276,31 +286,6 @@ namespace Lucene.Net.Spatial.Prefix
                 }
             }
 
-            /// <exception cref="System.IO.IOException"></exception>
-            public override DocIdSetIterator Iterator()
-            {
-                if (Size() == 0)
-                {
-                    return null;
-                }
-                //copy the unsorted values to a new array then sort them
-                int d = 0;
-                int[] docs = new int[intSet.Size];
-                foreach (int v in intSet.keys)
-                {
-                    if (v == intSet.emptyVal)
-                    {
-                        continue;
-                    }
-                    docs[d++] = v;
-                }
-                System.Diagnostics.Debug.Assert(d == intSet.Size);
-                int size = d;
-                //sort them
-                Array.Sort(docs, 0, size);
-                return new _DocIdSetIterator_225(size, docs);
-            }
-
             private sealed class _DocIdSetIterator_225 : DocIdSetIterator
             {
                 public _DocIdSetIterator_225(int size, int[] docs)
@@ -312,18 +297,15 @@ namespace Lucene.Net.Spatial.Prefix
 
                 internal int idx;
 
-                public override int DocID
+                public override int DocID()
                 {
-                    get
+                    if (this.idx >= 0 && this.idx < size)
                     {
-                        if (this.idx >= 0 && this.idx < size)
-                        {
-                            return docs[this.idx];
-                        }
-                        else
-                        {
-                            return -1;
-                        }
+                        return docs[this.idx];
+                    }
+                    else
+                    {
+                        return -1;
                     }
                 }
 
@@ -345,9 +327,9 @@ namespace Lucene.Net.Spatial.Prefix
                     return this.SlowAdvance(target);
                 }
 
-                public override long Cost
+                public override long Cost()
                 {
-                    get { return size; }
+                    return size;
                 }
 
                 private readonly int size;
@@ -359,6 +341,40 @@ namespace Lucene.Net.Spatial.Prefix
             {
                 get { return intSet.Exists(index); }
             }
+
+            public override DocIdSetIterator GetIterator()
+            {
+                if (Size() == 0)
+                {
+                    return null;
+                }
+                //copy the unsorted values to a new array then sort them
+                int d = 0;
+                int[] docs = new int[intSet.Size()];
+                foreach (int v in intSet.Keys)
+                {
+                    if (v == intSet.EmptyVal)
+                    {
+                        continue;
+                    }
+                    docs[d++] = v;
+                }
+                System.Diagnostics.Debug.Assert(d == intSet.Size());
+                int size = d;
+                //sort them
+                Array.Sort(docs, 0, size);
+                return new _DocIdSetIterator_225(size, docs);
+            }
+
+            public bool Get(int index)
+            {
+                throw new NotImplementedException();
+            }
+
+            int Bits.Length()
+            {
+                throw new NotImplementedException();
+            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4a485ee4/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
index 7201167..f5d8713 100644
--- a/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
@@ -33,10 +33,9 @@ namespace Lucene.Net.Spatial.Prefix
     {
         private readonly bool hasIndexedLeaves;
 
-        public IntersectsPrefixTreeFilter(Shape queryShape, string
-                                                                fieldName, SpatialPrefixTree grid, int detailLevel,
-                                          int prefixGridScanLevel, bool
-                                                                       hasIndexedLeaves)
+        public IntersectsPrefixTreeFilter(Shape queryShape, string fieldName, 
+                                          SpatialPrefixTree grid, int detailLevel,
+                                          int prefixGridScanLevel, bool hasIndexedLeaves)
             : base(queryShape, fieldName, grid, detailLevel, prefixGridScanLevel)
         {
             this.hasIndexedLeaves = hasIndexedLeaves;
@@ -48,8 +47,7 @@ namespace Lucene.Net.Spatial.Prefix
         }
 
         /// <exception cref="System.IO.IOException"></exception>
-        public override DocIdSet GetDocIdSet(AtomicReaderContext context, IBits acceptDocs
-            )
+        public override DocIdSet GetDocIdSet(AtomicReaderContext context, Bits acceptDocs)
         {
             return new _VisitorTemplate_55(this, context, acceptDocs, hasIndexedLeaves).GetDocIdSet
                 ();
@@ -62,9 +60,8 @@ namespace Lucene.Net.Spatial.Prefix
             private readonly IntersectsPrefixTreeFilter _enclosing;
             private FixedBitSet results;
 
-            public _VisitorTemplate_55(IntersectsPrefixTreeFilter _enclosing, AtomicReaderContext
-                                                                                  baseArg1, IBits baseArg2,
-                                       bool baseArg3)
+            public _VisitorTemplate_55(IntersectsPrefixTreeFilter _enclosing, AtomicReaderContext baseArg1, 
+                Bits baseArg2, bool baseArg3)
                 : base(_enclosing, baseArg1, baseArg2, baseArg3)
             {
                 this._enclosing = _enclosing;


[26/26] lucenenet git commit: Spatial: Updated to use the Spatial4n 0.4.1-beta NuGet package (closes #174)

Posted by ni...@apache.org.
Spatial: Updated to use the Spatial4n 0.4.1-beta NuGet package (closes #174)


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

Branch: refs/heads/master
Commit: e8735ed9622ab56eaf86bae6394f93f3fb058d30
Parents: 8219d87
Author: Shad Storhaug <sh...@shadstorhaug.com>
Authored: Fri Nov 25 17:56:37 2016 +0700
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Fri Nov 25 17:58:43 2016 +0700

----------------------------------------------------------------------
 .../Lucene.Net.Spatial.csproj                   |  8 ++++----
 .../Prefix/ContainsPrefixTreeFilter.cs          |  2 +-
 src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs      |  4 ++--
 .../Prefix/WithinPrefixTreeFilter.cs            |  6 +++---
 .../Serialized/SerializedDVStrategy.cs          |  2 +-
 src/Lucene.Net.Spatial/packages.config          |  2 +-
 .../Lucene.Net.Tests.Spatial.csproj             | 20 ++++++++++++--------
 .../Prefix/SpatialOpRecursivePrefixTreeTest.cs  | 15 ++++++++++++++-
 src/Lucene.Net.Tests.Spatial/packages.config    |  4 ++++
 9 files changed, 42 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/e8735ed9/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj b/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj
index 28c3e51..fdb56ba 100644
--- a/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj
+++ b/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj
@@ -57,10 +57,6 @@
     <StartupObject />
   </PropertyGroup>
   <ItemGroup>
-    <Reference Include="Spatial4n.Core.NTS">
-      <HintPath>..\..\packages\Spatial4n.Core.NTS.0.4.1\lib\net40\Spatial4n.Core.NTS.dll</HintPath>
-      <Private>True</Private>
-    </Reference>
     <Reference Include="GeoAPI, Version=1.7.4.0, Culture=neutral, PublicKeyToken=a1a0da7def465678, processorArchitecture=MSIL">
       <HintPath>..\..\packages\GeoAPI.1.7.4\lib\net45\GeoAPI.dll</HintPath>
       <Private>True</Private>
@@ -73,6 +69,10 @@
       <HintPath>..\..\packages\NetTopologySuite.1.14\lib\net45\PowerCollections.dll</HintPath>
       <Private>True</Private>
     </Reference>
+    <Reference Include="Spatial4n.Core, Version=0.4.1.0, Culture=neutral, PublicKeyToken=9f9456e1ca16d45e, processorArchitecture=MSIL">
+      <HintPath>..\..\packages\Spatial4n.Core.0.4.1-beta\lib\net40\Spatial4n.Core.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
     <Reference Include="System" />
     <Reference Include="System.Core" />
     <Reference Include="Microsoft.CSharp" />

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/e8735ed9/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
index 30bcba0..44da4c6 100644
--- a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
@@ -97,7 +97,7 @@ namespace Lucene.Net.Spatial.Prefix
                 SmallDocSet combinedSubResults = null;
                 //   Optimization: use null subCellsFilter when we know cell is within the query shape.
                 IShape subCellsFilter = outerInstance.queryShape;
-                if (cell.Level != 0 && ((cell.ShapeRel == SpatialRelation.NULL_VALUE || cell.ShapeRel == SpatialRelation.WITHIN)))
+                if (cell.Level != 0 && ((cell.ShapeRel == SpatialRelation.NOT_SET || cell.ShapeRel == SpatialRelation.WITHIN)))
                 {
                     subCellsFilter = null;
                     Debug.Assert(cell.Shape.Relate(outerInstance.queryShape) == SpatialRelation.WITHIN);

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/e8735ed9/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs b/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
index 91db4d5..2232185 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
@@ -58,7 +58,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// When set via <see cref="GetSubCells(IShape)">GetSubCells(filter)</see>, it is the relationship between this cell
         /// and the given shape filter.
         /// </summary>
-        protected internal SpatialRelation shapeRel = SpatialRelation.NULL_VALUE;//set in GetSubCells(filter), and via SetLeaf().
+        protected internal SpatialRelation shapeRel;//set in GetSubCells(filter), and via SetLeaf().
 
         /// <summary>Always false for points.</summary>
         /// <remarks>
@@ -105,7 +105,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         {
             Debug.Assert(Level != 0);
             token = null;
-            shapeRel = SpatialRelation.NULL_VALUE;
+            shapeRel = SpatialRelation.NOT_SET;
             this.bytes = bytes;
             b_off = off;
             b_len = len;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/e8735ed9/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
index 0cf7b6e..6a7a8f0 100644
--- a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
@@ -219,7 +219,7 @@ namespace Lucene.Net.Spatial.Prefix
             /// </summary>
             private bool AllCellsIntersectQuery(Cell cell, SpatialRelation relate/*cell to query*/)
             {
-                if (relate == SpatialRelation.NULL_VALUE)
+                if (relate == SpatialRelation.NOT_SET)
                 {
                     relate = cell.Shape.Relate(outerInstance.queryShape);
                 }
@@ -241,7 +241,7 @@ namespace Lucene.Net.Spatial.Prefix
                 ICollection<Cell> subCells = cell.GetSubCells(null);
                 foreach (Cell subCell in subCells)
                 {
-                    if (!AllCellsIntersectQuery(subCell, SpatialRelation.NULL_VALUE))
+                    if (!AllCellsIntersectQuery(subCell, SpatialRelation.NOT_SET))
                     {
                         //recursion
                         return false;
@@ -253,7 +253,7 @@ namespace Lucene.Net.Spatial.Prefix
             /// <exception cref="System.IO.IOException"></exception>
             protected internal override void VisitScanned(Cell cell)
             {
-                if (AllCellsIntersectQuery(cell, SpatialRelation.NULL_VALUE))
+                if (AllCellsIntersectQuery(cell, SpatialRelation.NOT_SET))
                 {
                     CollectDocs(inside);
                 }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/e8735ed9/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
index e401c34..6d6e185 100644
--- a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
+++ b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
@@ -7,7 +7,7 @@ using Lucene.Net.Spatial.Util;
 using Lucene.Net.Support;
 using Lucene.Net.Util;
 using Spatial4n.Core.Context;
-using Spatial4n.Core.Io;
+using Spatial4n.Core.IO;
 using Spatial4n.Core.Shapes;
 using System;
 using System.Collections;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/e8735ed9/src/Lucene.Net.Spatial/packages.config
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/packages.config b/src/Lucene.Net.Spatial/packages.config
index 16ef20c..b55d4fc 100644
--- a/src/Lucene.Net.Spatial/packages.config
+++ b/src/Lucene.Net.Spatial/packages.config
@@ -1,6 +1,6 @@
 \ufeff<?xml version="1.0" encoding="utf-8"?>
 <packages>
-  <!--<package id="Spatial4n.Core.NTS" version="0.4.1" targetFramework="net451" />-->
   <package id="GeoAPI" version="1.7.4" targetFramework="net451" />
   <package id="NetTopologySuite" version="1.14" targetFramework="net451" />
+  <package id="Spatial4n.Core" version="0.4.1-beta" targetFramework="net451" />
 </packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/e8735ed9/src/Lucene.Net.Tests.Spatial/Lucene.Net.Tests.Spatial.csproj
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Lucene.Net.Tests.Spatial.csproj b/src/Lucene.Net.Tests.Spatial/Lucene.Net.Tests.Spatial.csproj
index d74a255..95a21f6 100644
--- a/src/Lucene.Net.Tests.Spatial/Lucene.Net.Tests.Spatial.csproj
+++ b/src/Lucene.Net.Tests.Spatial/Lucene.Net.Tests.Spatial.csproj
@@ -30,14 +30,6 @@
     <WarningLevel>4</WarningLevel>
   </PropertyGroup>
   <ItemGroup>
-    <Reference Include="nunit.framework, Version=2.6.3.13283, Culture=neutral, PublicKeyToken=96d09a1eb7f44a77, processorArchitecture=MSIL">
-      <HintPath>..\..\packages\NUnit.2.6.3\lib\nunit.framework.dll</HintPath>
-      <Private>True</Private>
-    </Reference>
-    <Reference Include="Spatial4n.Core.NTS">
-      <HintPath>..\..\packages\Spatial4n.Core.NTS.0.4.1\lib\net40\Spatial4n.Core.NTS.dll</HintPath>
-      <Private>True</Private>
-    </Reference>
     <Reference Include="GeoAPI, Version=1.7.4.0, Culture=neutral, PublicKeyToken=a1a0da7def465678, processorArchitecture=MSIL">
       <HintPath>..\..\packages\GeoAPI.1.7.4\lib\net45\GeoAPI.dll</HintPath>
       <Private>True</Private>
@@ -46,10 +38,22 @@
       <HintPath>..\..\packages\NetTopologySuite.1.14\lib\net45\NetTopologySuite.dll</HintPath>
       <Private>True</Private>
     </Reference>
+    <Reference Include="nunit.framework, Version=2.6.3.13283, Culture=neutral, PublicKeyToken=96d09a1eb7f44a77, processorArchitecture=MSIL">
+      <HintPath>..\..\packages\NUnit.2.6.3\lib\nunit.framework.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
     <Reference Include="PowerCollections, Version=1.0.0.0, Culture=neutral, PublicKeyToken=2573bf8a1bdddcd5, processorArchitecture=MSIL">
       <HintPath>..\..\packages\NetTopologySuite.1.14\lib\net45\PowerCollections.dll</HintPath>
       <Private>True</Private>
     </Reference>
+    <Reference Include="Spatial4n.Core, Version=0.4.1.0, Culture=neutral, PublicKeyToken=9f9456e1ca16d45e, processorArchitecture=MSIL">
+      <HintPath>..\..\packages\Spatial4n.Core.0.4.1-beta\lib\net40\Spatial4n.Core.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="Spatial4n.Core.NTS, Version=0.4.1.0, Culture=neutral, PublicKeyToken=9f9456e1ca16d45e, processorArchitecture=MSIL">
+      <HintPath>..\..\packages\Spatial4n.Core.NTS.0.4.1-beta\lib\net40\Spatial4n.Core.NTS.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
     <Reference Include="System" />
     <Reference Include="System.Core" />
     <Reference Include="System.Xml.Linq" />

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/e8735ed9/src/Lucene.Net.Tests.Spatial/Prefix/SpatialOpRecursivePrefixTreeTest.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Prefix/SpatialOpRecursivePrefixTreeTest.cs b/src/Lucene.Net.Tests.Spatial/Prefix/SpatialOpRecursivePrefixTreeTest.cs
index d53f4a8..7bf6041 100644
--- a/src/Lucene.Net.Tests.Spatial/Prefix/SpatialOpRecursivePrefixTreeTest.cs
+++ b/src/Lucene.Net.Tests.Spatial/Prefix/SpatialOpRecursivePrefixTreeTest.cs
@@ -58,7 +58,7 @@ namespace Lucene.Net.Spatial.Prefix
         {
             //non-geospatial makes this test a little easier (in gridSnap), and using boundary values 2^X raises
             // the prospect of edge conditions we want to test, plus makes for simpler numbers (no decimals).
-            SpatialContextFactory factory = new SpatialContextFactory();
+            FakeSpatialContextFactory factory = new FakeSpatialContextFactory();
             factory.geo = false;
             factory.worldBounds = new Rectangle(0, 256, -128, 128, null);
             this.ctx = factory.NewSpatialContext();
@@ -69,6 +69,19 @@ namespace Lucene.Net.Spatial.Prefix
             this.strategy = new RecursivePrefixTreeStrategy(grid, GetType().Name);
         }
 
+        /// <summary>
+        /// LUCENENET specific class used to gain access to protected internal
+        /// member NewSpatialContext(), since we are not strong-named and
+        /// InternalsVisibleTo is not an option from a strong-named class.
+        /// </summary>
+        private class FakeSpatialContextFactory : SpatialContextFactory
+        {
+            new public SpatialContext NewSpatialContext()
+            {
+                return base.NewSpatialContext();
+            }
+        }
+
         public virtual void SetupGeohashGrid(int maxLevels)
         {
             this.ctx = SpatialContext.GEO;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/e8735ed9/src/Lucene.Net.Tests.Spatial/packages.config
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/packages.config b/src/Lucene.Net.Tests.Spatial/packages.config
index 139d513..b28dff1 100644
--- a/src/Lucene.Net.Tests.Spatial/packages.config
+++ b/src/Lucene.Net.Tests.Spatial/packages.config
@@ -1,4 +1,8 @@
 \ufeff<?xml version="1.0" encoding="utf-8"?>
 <packages>
+  <package id="GeoAPI" version="1.7.4" targetFramework="net451" />
+  <package id="NetTopologySuite" version="1.14" targetFramework="net451" />
   <package id="NUnit" version="2.6.3" targetFramework="net451" />
+  <package id="Spatial4n.Core" version="0.4.1-beta" targetFramework="net451" />
+  <package id="Spatial4n.Core.NTS" version="0.4.1-beta" targetFramework="net451" />
 </packages>
\ No newline at end of file


[13/26] lucenenet git commit: Added missing Spatial classes, ported Spatial tests, and fixed several bugs

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Test-Files/Data/countries-poly.txt
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Test-Files/Data/countries-poly.txt b/src/Lucene.Net.Tests.Spatial/Test-Files/Data/countries-poly.txt
new file mode 100644
index 0000000..8192df2
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Test-Files/Data/countries-poly.txt
@@ -0,0 +1,249 @@
+#id	name	shape	
+FLK	Falkland Is.	MULTIPOLYGON (((-59.348063887634126 -52.34305496492924, -59.37944588766335 -52.32778196491501, -59.57444588784496 -52.21528196481024, -59.71611788797689 -52.11737296471905, -59.56667288783771 -51.91749996453291, -59.51805488779243 -51.87528196449359, -59.2883358875785 -51.746944964374066, -59.221254887516025 -51.71709096434626, -59.13306388743389 -51.69444496432517, -59.055554887361694 -51.691672964322585, -59.03087288733872 -51.7643819643903, -59.04139088734851 -51.81520896443764, -59.00083588731074 -51.8100089644328, -58.99472688730505 -51.80667296442969, -58.9826458872938 -51.79534496441914, -59.02389988733222 -51.67694496430887, -59.07833588738292 -51.631954964266974, -59.168263887466665 -51.585281964223505, -59.10860888741111 -51.513890964157014, -59.08416388738834 -51.53722696417875, -59.06986388737502 -51.557363964197506, -59.039308887346564 -51.58396396422228, -59.01430888732328 -51.575835964214704, -58.989445887300135 -51.50417296414796, -59.0461088873529 -
 51.47889096412442, -59.074726887379555 -51.469863964116016, -59.08472688738887 -51.41166396406181, -58.99110888730168 -51.40555496405612, -58.868754887187734 -51.37007296402307, -58.88430888720221 -51.33083596398653, -58.84861788716897 -51.291390963949794, -58.75403588708089 -51.32639096398239, -58.69667288702746 -51.336672963991965, -58.614508886950944 -51.3294089639852, -58.57972688691855 -51.30944496396661, -58.5444458868857 -51.30569996396312, -58.4673638868139 -51.307363963964676, -58.41250888676282 -51.32340896397962, -58.34361788669865 -51.36903596402211, -58.32459088668094 -51.416081964065924, -58.36194588671573 -51.44445496409235, -58.44389088679205 -51.45361796410088, -58.45917288680627 -51.43020896407908, -58.47597288682192 -51.39999996405095, -58.49610888684067 -51.39631796404752, -58.55028188689113 -51.43361796408225, -58.51833588686138 -51.47610896412183, -58.49889088684327 -51.50028196414434, -58.420563886770324 -51.556672964196856, -58.36249988671625 -51.567217964206
 684, -58.33583588669141 -51.56667296420617, -58.28167288664096 -51.606808964243555, -58.241390886603455 -51.65014496428391, -58.21743588658114 -51.65069996428443, -58.18278188654887 -51.60917296424575, -58.188899886554566 -51.585281964223505, -58.2572268866182 -51.53055496417254, -58.31409988667117 -51.50041796414447, -58.345554886700455 -51.509099964152554, -58.27479988663457 -51.413754964063756, -58.25028188661173 -51.40097296405185, -58.21736388658107 -51.39528196404655, -57.94445488632691 -51.371808964024694, -57.91541788629986 -51.37528196402793, -57.89680888628253 -51.3841729640362, -57.873890886261194 -51.4016729640525, -57.768054886162616 -51.50403596414784, -57.77291788616715 -51.54375496418483, -57.81389988620532 -51.54972696419039, -57.87861788626559 -51.54610896418702, -57.90472688628991 -51.543335964184436, -57.93028188631371 -51.539999964181334, -57.95360888633543 -51.53486396417655, -57.9794458863595 -51.52389096416633, -58.00417288638252 -51.50778196415133, -58.03541
 788641162 -51.50209096414603, -58.138963886508066 -51.54986396419052, -58.109308886480434 -51.570563964209796, -58.039235886415184 -51.59014496422803, -58.0087548863868 -51.59319996423088, -57.983545886363316 -51.58805496422609, -57.87694588626404 -51.60000896423722, -57.79764588619018 -51.61180896424821, -57.77528188616935 -51.621735964257454, -57.733199886130166 -51.69444496432517, -57.83444588622446 -51.723617964352336, -57.973890886354326 -51.74860896437561, -58.0324998864089 -51.7575089643839, -58.1119458864829 -51.76333596438933, -58.19778188656284 -51.761944964388036, -58.249172886610694 -51.7573999643838, -58.21389088657784 -51.78110896440588, -58.17361788654033 -51.796672964420374, -58.200008886564916 -51.793890964417784, -58.24680888660849 -51.79333596441727, -58.35833588671237 -51.82666396444831, -58.33778188669322 -51.83278196445401, -58.31514588667214 -51.83444496445556, -58.28528188664433 -51.82722696444883, -58.234726886597244 -51.83278196445401, -58.39291788674457 -5
 1.89583596451273, -58.4199998867698 -51.90069996451726, -58.60639088694339 -51.90028196451687, -58.76812688709401 -51.89125496450846, -58.88945488720701 -51.84472696446513, -58.90917288722537 -51.83583596445685, -58.92347288723869 -51.820281964442366, -58.935554887249936 -51.80125496442464, -58.96243588727498 -51.81764496443991, -58.97382688728558 -51.85146396447141, -58.926672887241665 -51.87806396449618, -58.82389088714595 -51.91528196453084, -58.720626887049775 -51.94784496456116, -58.63396388696907 -51.96451796457669, -58.60389088694106 -52.001399964611046, -58.64694588698116 -52.067217964672345, -58.6802818870122 -52.090835964694335, -58.734726887062905 -52.046672964653204, -58.78791788711244 -52.03417296464156, -58.82500888714699 -52.047226964653724, -58.856526887176344 -52.0688909646739, -58.918890887234426 -52.099999964702874, -58.981108887292365 -52.06958196467454, -59.03333588734101 -52.02444496463251, -59.14236388744254 -51.985281964596034, -59.251672887544345 -51.9911089
 6460146, -59.29430888758405 -52.0097269646188, -59.279654887570416 -52.02493596463297, -59.24444588753762 -52.01944496462785, -59.21959088751447 -52.0234729646316, -59.19180888748859 -52.029444964637165, -59.12222688742379 -52.063335964668724, -59.03625488734373 -52.1408359647409, -59.05069988735718 -52.217499964812305, -59.0720908873771 -52.231535964825376, -59.19389088749054 -52.206672964802216, -59.24083588753426 -52.18639096478333, -59.29249988758238 -52.157499964756425, -59.37951788766341 -52.11923596472079, -59.44979088772887 -52.1467359647464, -59.428408887708954 -52.21389096480894, -59.38444588766801 -52.22722696482136, -59.343472887629844 -52.25215496484458, -59.33805488762479 -52.324444964911905, -59.348063887634126 -52.34305496492924)), ((-60.34528188856285 -51.86083596448013, -60.310835888530775 -51.81139096443408, -60.25749988848111 -51.77486396440007, -60.222781888448765 -51.7877819644121, -60.18284588841158 -51.75785496438422, -60.178481888407504 -51.71222696434173, -
 60.201117888428584 -51.70333596433345, -60.22749988845317 -51.70028196433061, -60.35610888857293 -51.711317964340886, -60.38556388860037 -51.744726964372, -60.445281888655984 -51.76249996438855, -60.5561178887592 -51.77555496440071, -60.63437288883209 -51.72499996435363, -60.638408888835855 -51.67972696431146, -60.61389088881302 -51.671108964303436, -60.57722688877887 -51.68528196431664, -60.55722688876024 -51.69306396432388, -60.52472688872997 -51.69944496432983, -60.48861788869634 -51.702781964332935, -60.45722688866711 -51.70222696433242, -60.43625488864758 -51.697090964327636, -60.39939088861324 -51.67999996431172, -60.36472688858096 -51.66749996430008, -60.34389088856156 -51.66528196429801, -60.248890888473085 -51.665554964298266, -60.163608888393654 -51.67111796430344, -60.29444588851551 -51.59861796423592, -60.40444588861796 -51.55139096419194, -60.390626888605084 -51.4922909641369, -60.40391788861746 -51.46865496411488, -60.448126888658635 -51.44507296409292, -60.50555488871
 211 -51.43652696408496, -60.55500888875818 -51.437435964085815, -60.62201788882058 -51.41152696406168, -60.643617888840694 -51.358054964011885, -60.60694588880655 -51.34861796400309, -60.48360888869168 -51.38528196403724, -60.39744588861143 -51.42349996407283, -60.333617888551984 -51.45555496410269, -60.3133358885331 -51.46333596410993, -60.25583588847955 -51.48111796412649, -60.22264588844864 -51.486672964131664, -60.12722688835977 -51.49472696413917, -59.992363888234166 -51.46562696411207, -60.04291788828125 -51.45069996409816, -60.0624998882995 -51.42083596407035, -60.023199888262894 -51.38125496403349, -59.89417288814272 -51.37236396402521, -59.87028188812047 -51.38110896403335, -59.825835888079084 -51.40722696405768, -59.82389088807727 -51.43322696408189, -59.777781888034326 -51.44694496409467, -59.750835888009235 -51.44917296409674, -59.5136178877883 -51.46194496410864, -59.45861788773708 -51.45806396410502, -59.39535488767817 -51.43104496407986, -59.41667288769801 -51.4141729
 6406414, -59.442154887721756 -51.40479096405541, -59.44889088772803 -51.35833596401214, -59.399799887682306 -51.33764496399287, -59.21167288750709 -51.40819996405858, -59.39833588768094 -51.60264496423967, -59.431945887712246 -51.62333596425894, -59.454235887733006 -51.62104496425681, -59.57778188784806 -51.68833596431948, -59.67361788793731 -51.771663964397085, -59.83167288808451 -51.8933359645104, -59.93194588817791 -51.968608964580504, -59.953054888197556 -51.98306396459397, -60.06486388830169 -51.940563964554386, -60.18999988841824 -51.97833596458956, -60.27250888849508 -52.04361796465036, -60.26451788848763 -52.07089996467577, -60.280554888502564 -52.09611796469925, -60.31444588853414 -52.1294449647303, -60.368054888584055 -52.15917296475798, -60.598617888798785 -52.24222696483533, -60.62458188882297 -52.243199964836236, -60.657363888853496 -52.22778196482188, -60.73528188892607 -52.17778196477531, -60.86278188904481 -52.119444964720984, -60.98083588915476 -52.06195496466744, -
 60.91111788908982 -52.03110896463871, -60.82020888900516 -52.033754964641176, -60.75041788894016 -52.00659996461589, -60.813408888998836 -51.99264496460289, -60.84403588902735 -51.9619449645743, -60.76611788895478 -51.95583596456861, -60.70722688889994 -51.96167296457404, -60.67583588887071 -51.968890964580766, -60.64110888883836 -51.975008964586465, -60.53389088873851 -51.97749996458879, -60.45972688866944 -51.97083596458258, -60.43972688865081 -51.96208196457442, -60.40649988861986 -51.92409996453905, -60.42444588863658 -51.911944964527734, -60.44944588865987 -51.88305496450083, -60.45403588866414 -51.80583596442891, -60.44034588865139 -51.782844964407495, -60.38125488859636 -51.76653596439231, -60.363199888579544 -51.8064639644295, -60.37610888859156 -51.83459096445569, -60.34528188856285 -51.86083596448013)), ((-60.99305488916613 -51.96583596457792, -61.02944588920003 -51.94722696456059, -61.112154889277065 -51.89541796451234, -61.14805488931049 -51.84639096446668, -61.139163889
 30221 -51.835281964456335, -61.12264588928683 -51.821390964443395, -61.01306388918478 -51.779726964404595, -60.99917288917183 -51.77847296440343, -60.96805488914285 -51.78110896440588, -60.938545889115375 -51.80680896442982, -60.945554889121894 -51.81917296444133, -60.95667288913225 -51.826117964447796, -60.994172889167174 -51.83861796445944, -61.01389088918555 -51.84806396446824, -61.03076388920125 -51.86590896448486, -61.01639988918788 -51.869726964488414, -60.97278188914726 -51.86028196447962, -60.94944588912553 -51.853335964473146, -60.92694588910457 -51.84528196446565, -60.9072268890862 -51.836390964457365, -60.89083588907094 -51.825835964447535, -60.87528188905645 -51.83999996446073, -60.86847288905011 -51.906108964522296, -60.88110888906188 -51.93444496454869, -60.91222688909086 -51.94389096455748, -60.96805488914285 -51.95972696457223, -60.99305488916613 -51.96583596457792)), ((-60.28526388850696 -51.37564496402826, -60.27944588850154 -51.36361796401706, -60.28917288851059 -
 51.28583596394462, -60.299172888519905 -51.274444963934016, -60.29097288851227 -51.26652696392664, -60.273199888495725 -51.26514496392535, -60.07944588831528 -51.30083596395859, -60.06806388830468 -51.3074999639648, -60.0483358882863 -51.33333596398886, -60.05278188829044 -51.34667296400128, -60.11360888834709 -51.40680896405729, -60.24986388847398 -51.403199964053925, -60.268335888491194 -51.39805496404914, -60.285135888506844 -51.38403596403608, -60.28526388850696 -51.37564496402826)), ((-59.791390888047005 -51.24945496391074, -59.73194588799164 -51.2530639639141, -59.58361788785349 -51.25750896391824, -59.48319988775998 -51.26389096392418, -59.47306388775054 -51.26778196392781, -59.45083588772984 -51.30555496396299, -59.44555488772491 -51.320281963976704, -59.467781887745616 -51.335281963990674, -59.50833588778339 -51.33749996399274, -59.53833588781133 -51.33749996399274, -59.55749988782918 -51.336672963991965, -59.57361788784418 -51.33361796398912, -59.802363888057215 -51.274590
 96393415, -59.81110888806536 -51.26972696392962, -59.80514588805981 -51.25499996391591, -59.791390888047005 -51.24945496391074)))	
+GUF	French Guiana	MULTIPOLYGON (((-54.60378188321566 2.3291910859882705, -54.499172883118234 2.3650000860216096, -54.420563883045034 2.434718086086548, -54.365908882994134 2.49660008614417, -54.35195488298113 2.5238910861695985, -54.32180888295305 2.5984730862390535, -54.31513588294685 2.629164086267636, -54.25695488289266 2.7188910863512064, -54.222917882860955 2.755554086385345, -54.20459088284389 2.775000086403452, -54.19528188283522 2.796245086423241, -54.16083588280314 2.9536090865697986, -54.176117882817366 3.043891086653886, -54.19574588283565 3.096200086702595, -54.2084728828475 3.124309086728772, -54.206117882845305 3.14528208674831, -54.187226882827716 3.1948640867944818, -54.11875488276395 3.27778208687171, -54.09445488274132 3.2950000868877396, -54.07110888271957 3.3127820869043063, -54.00110888265438 3.4483360870305546, -53.986117882640414 3.601527087173224, -53.99778188265128 3.626945087196887, -54.12764588277223 3.788609087347453, -54.14639088278969 3.7977820873559978
 , -54.18868188282907 3.808745087366205, -54.29028188292369 3.9338910874827633, -54.36076388298933 4.042427087583846, -54.365008882993294 4.163609087696699, -54.39389088302019 4.237218087765257, -54.43139988305512 4.369164087888137, -54.44023588306335 4.410645087926767, -54.45180888307412 4.512364088021499, -54.442917883065846 4.530691088038566, -54.42326388304754 4.5643090880698765, -54.42209088304645 4.601527088104547, -54.439726883062875 4.669164088167534, -54.47333588309418 4.734582088228464, -54.47778188309832 4.7541640882466965, -54.47610888309676 4.867773088352507, -54.45528188307736 5.004027088479404, -54.448063883070645 5.0241640884981535, -54.4086178830339 5.08055408855067, -54.37889088300622 5.112218088580164, -54.347781882977245 5.148336088613803, -54.317326882948876 5.208627088669942, -54.28348188291736 5.254864088713006, -54.244999882881515 5.2850000887410715, -54.19333588283341 5.315273088769274, -54.17069988281233 5.342218088794368, -54.16668188280859 5.34740008879919
 1, -54.1398638827836 5.357782088808861, -54.091663882738715 5.3919450888406715, -54.06959088271816 5.4180540888649915, -54.058199882707555 5.437364088882973, -54.006735882659626 5.5447910889830325, -54.003890882656975 5.575691089011798, -54.0111178826637 5.605973089040006, -54.00569988265866 5.641527089073122, -53.99222688264611 5.673054089102479, -53.981672882636275 5.690554089118777, -53.93972688259721 5.7447180891692255, -53.91139088257083 5.750282089174405, -53.858608882521665 5.755418089179187, -53.7508358824213 5.731391089156816, -53.63667288231497 5.673054089102479, -53.52236388220851 5.60471808903884, -53.49917288218691 5.58028208901608, -53.49433588218241 5.572345089008692, -53.48360888217242 5.56805408900469, -53.40833588210232 5.548891088986849, -53.30278188200401 5.52305408896278, -53.18639088189562 5.499164088940532, -53.084445881800676 5.483336088925796, -52.97319988169707 5.473054088916214, -52.93778188166408 5.458336088902513, -52.88652688161635 5.420282088867069, -5
 2.799726881535506 5.341945088794105, -52.78750888152413 5.321945088775479, -52.736390881476524 5.260554088718308, -52.611117881359846 5.129445088596199, -52.56750888131924 5.096664088565674, -52.42222688118393 4.99250008846866, -52.330699881098695 4.948891088428056, -52.28972688106053 4.938400088418277, -52.06444588085073 4.733891088227821, -52.02347288081256 4.685691088182921, -51.99694588078785 4.6430540881432165, -51.985281880776995 4.613609088115794, -51.97805488077026 4.586945088090957, -51.95625488074995 4.492364088002873, -51.950835880744904 4.456664087969628, -51.950835880744904 4.423891087939111, -51.96041788075385 4.400136087916977, -52.001254880791876 4.368891087887889, -52.029035880817744 4.352364087872488, -52.04069988082861 4.334582087855935, -51.99194588078319 4.348191087868599, -51.951181880745224 4.372636087891365, -51.92819988072384 4.400836087917639, -51.923617880719576 4.4270820879420825, -51.92639088072215 4.465282087977656, -51.929308880724875 4.486809087997699
 , -51.931399880726815 4.529164088037149, -51.931399880726815 4.570000088075176, -51.92444588072033 4.620973088122653, -51.91555488071205 4.646527088146456, -51.900008880697584 4.66180908816068, -51.8615998806618 4.659309088158352, -51.794445880599255 4.605554088108292, -51.76701788057372 4.5376360880450335, -51.75917288056641 4.500282088010252, -51.75639088056383 4.477218087988774, -51.75249988056021 4.455282087968342, -51.74417288055244 4.4211090879365145, -51.71389088052425 4.3129180878357545, -51.70249988051364 4.287082087811697, -51.6744458804875 4.253336087780269, -51.65569988047005 4.225418087754264, -51.648472880463316 4.200282087730855, -51.64805488046292 4.167218087700064, -51.65222688046683 4.1372180876721245, -51.66104588047503 4.081109087619865, -51.6744458804875 4.049445087590371, -51.68406388049647 4.034164087576144, -51.70417288051519 4.026109087568642, -51.728335880537685 4.01583608755908, -51.7538908805615 4.000554087544842, -51.77180888057819 3.9834730875289353, -5
 1.78361788058919 3.961109087508106, -51.79389088059875 3.916664087466714, -51.81232688061593 3.876818087429612, -51.906390880703526 3.7908360873495326, -51.927081880722795 3.7769450873365855, -51.98104588077305 3.7000000872649252, -51.9900088807814 3.657500087225344, -51.995835880786814 3.6258360871958644, -52.1002818808841 3.448891087031072, -52.16528188094463 3.342218086931723, -52.226745881001875 3.253054086848678, -52.24965488102322 3.2437450868400077, -52.27360888104552 3.2410450868375023, -52.34493588111195 3.157400086759594, -52.353054881119505 3.1316640867356256, -52.34486388111188 3.0884730866954015, -52.35195488111849 3.0400000866502523, -52.37917288114383 2.9741640865889423, -52.40861788117125 2.9222180865405676, -52.46417288122299 2.8133360864391648, -52.519726881274735 2.697782086331543, -52.55278188130552 2.6212450862602594, -52.539863881293485 2.586391086227806, -52.54222688129569 2.565554086208394, -52.563754881315745 2.522082086167913, -52.59445488134433 2.473891086
 1230323, -52.67562688141993 2.3741640860301487, -52.78945488152594 2.3033360859641903, -52.88278188161286 2.22471808589097, -52.89646388162561 2.2068090858742835, -52.95472688167986 2.176182085845767, -52.99430888171672 2.175691085845301, -53.054863881773116 2.1869450858557826, -53.06959088178684 2.2030540858707894, -53.087363881803384 2.2194450858860506, -53.11180888182615 2.222500085888896, -53.196108881904664 2.21360908588062, -53.23917288194477 2.2083360858757146, -53.2261088819326 2.26444508592796, -53.296535881998196 2.3190090859787773, -53.319863882019916 2.3479820860057714, -53.340563882039206 2.3496540860073196, -53.34833588204644 2.3227090859822255, -53.45861788214914 2.2575000859214924, -53.52778188221356 2.249509085914056, -53.7016638823755 2.3102820859706554, -53.72153588239401 2.3417360859999548, -53.74610888241689 2.3709730860271776, -53.77680888244548 2.3645820860212297, -53.79944588246657 2.3523640860098425, -53.91028188256979 2.2772180859398645, -53.92806388258636 
 2.2527820859171044, -53.93263588259062 2.2280540858940725, -54.10965488275548 2.113473085787362, -54.32055488295188 2.1615270858321196, -54.4646548830861 2.211736085878883, -54.51861788313634 2.2575000859214924, -54.527226883144365 2.2893090859511176, -54.541326883157495 2.3152090859752406, -54.57389088318783 2.3252820859846253, -54.59361788320619 2.329718085988759, -54.60378188321566 2.3291910859882705)))	
+GUY	Guyana	MULTIPOLYGON (((-58.1726178865394 6.812218090163412, -58.154935886522935 6.828191090178279, -58.03889088641486 6.815554090166515, -57.98639088636597 6.790554090143232, -57.966672886347595 6.775418090129136, -57.94333588632587 6.750554090105979, -57.92778188631138 6.731664090088387, -57.91445488629897 6.711109090069243, -57.897917886283565 6.689864090049454, -57.88250888626922 6.674164090034836, -57.757781886153055 6.570000089937821, -57.64069988604402 6.485136089858784, -57.5943088860008 6.4345820898117125, -57.5213908859329 6.290345089677373, -57.519190885930854 6.270764089659139, -57.498617885911685 6.338682089722397, -57.469717885884776 6.34027308972388, -57.363617885785956 6.290000089677051, -57.33667288576086 6.2747180896628265, -57.261117885690496 6.2113910896038504, -57.22028188565247 6.169164089564518, -57.194308885628274 6.1393090895367095, -57.17778188561289 6.108336089507873, -57.16236388559852 6.056945089460001, -57.13576388557375 5.954100089364232, -57.175281
 88561056 5.637500089069363, -57.182781885617544 5.600691089035081, -57.195563885629454 5.568891089005476, -57.24850888567876 5.486109088928373, -57.27159988570027 5.384864088834078, -57.2994458857262 5.359718088810666, -57.32444588574948 5.303609088758407, -57.282781885710676 5.226391088686498, -57.26222688569153 5.22166408868209, -57.23520888566637 5.259945088717743, -57.20750888564058 5.223609088683901, -57.19104588562524 5.172009088635846, -57.231526885662944 5.146454088612046, -57.25055488568066 5.172636088636423, -57.27209088570072 5.177082088640574, -57.292363885719595 5.166245088630475, -57.321945885747155 5.075973088546405, -57.32722688575207 5.026109088499965, -57.41709088583576 4.989445088465814, -57.4736178858884 4.989164088465557, -57.61098188601633 4.992218088468405, -57.63417288603793 5.000282088475913, -57.67625488607713 5.011527088486389, -57.84111788623066 4.927782088408392, -57.901390886286805 4.855554088341123, -57.92333588630724 4.821945088309818, -57.92486388630
 866 4.796391088286029, -57.909026886293915 4.777782088268694, -57.88778188627413 4.7683360882599, -57.84035488622996 4.669027088167411, -57.8469458862361 4.6288910881300325, -57.902781886288096 4.503609088013349, -57.94861788633078 4.349164087869511, -57.9491728863313 4.317218087839763, -57.954517886336276 4.288336087812866, -58.00278188638123 4.239718087767585, -58.039172886415116 4.211391087741205, -58.0618088864362 4.180136087712086, -58.071399886445136 4.15569108768932, -58.04375488641939 4.001527087545753, -58.01639988639391 3.9636090875104344, -57.96889088634967 3.9291640874783553, -57.95041788633246 3.913191087463474, -57.93194588631526 3.8906910874425193, -57.876672886263776 3.8091640873665966, -57.86749988625523 3.7908360873495326, -57.85945488624775 3.771664087331672, -57.84930888623829 3.7388910873011554, -57.84597288623519 3.692636087258066, -57.83848188622821 3.6693090872363427, -57.8166728862079 3.6502820872186277, -57.76056388615565 3.624164087194302, -57.740835886137
 276 3.613745087184597, -57.66861788607001 3.5383360871143736, -57.65444588605682 3.518609087095996, -57.63840888604189 3.457009087038628, -57.65778188605992 3.4138910869984755, -57.642226886045435 3.3563910869449245, -57.61041788601581 3.3600000869482756, -57.53055488594143 3.3463910869356113, -57.42945488584728 3.3472180869363797, -57.304172885730594 3.380418086967296, -57.28354588571139 3.318682086909803, -57.29249988571972 3.2673640868620026, -57.28999988571739 3.181391086781943, -57.28361788571145 3.140973086744296, -57.255008885684816 3.103609086709497, -57.22028188565247 3.0650000866735354, -57.210281885643155 3.035836086646384, -57.207217885640304 3.004236086616956, -57.213335885646 2.878891086500218, -57.20083588563436 2.8228450864480124, -57.11688188555617 2.757191086386868, -56.94472688539584 2.4830540861315598, -56.917781885370744 2.428054086080337, -56.853890885311245 2.283336085945564, -56.810563885270895 2.1933360858617448, -56.77499988523776 2.1408360858128503, -56.68
 209088515124 2.0267640857066027, -56.64611788511773 2.028054085707808, -56.579726885055905 2.0273640857071626, -56.55597288503378 2.020554085700823, -56.47736388496057 1.9620820856463723, -56.470635884954305 1.944500085629997, -56.52555488500545 1.9225000856094994, -56.57791788505422 1.9216640856087253, -56.59499988507012 1.9322180856185582, -56.628199885101054 1.9394450856252803, -56.73500888520053 1.9141000856016746, -56.75209088521643 1.896945085585699, -56.77069088523376 1.8756270855658528, -56.87389088532987 1.8916640855807856, -57.00389088545094 1.9229180856098935, -57.050835885494664 1.9522180856371847, -57.06083588550398 1.984164085666933, -57.07423588551646 2.015764085696361, -57.10020888554064 2.0227820857029, -57.163063885599186 1.9966640856785745, -57.199172885632805 1.9711090856547742, -57.235345885666504 1.9497910856349279, -57.27667288570498 1.979509085662599, -57.298190885725035 1.9819450856648615, -57.33194588575647 1.9722180856558111, -57.42549088584359 1.901391085
 5898501, -57.42805488584597 1.8712450855617675, -57.450908885867264 1.8111090855057626, -57.512363885924486 1.730836085431008, -57.52806388593912 1.7158360854170382, -57.55930888596821 1.6997180854020257, -57.63333588603716 1.6927820853955637, -57.65361788605604 1.6966640853991777, -57.68749988608759 1.707500085409265, -57.72249988612019 1.7175000854185782, -57.75944588615461 1.717845085418901, -57.87610888626325 1.6650000853696838, -57.97249988635302 1.6134730853217008, -57.98291788636273 1.5723640852834109, -58.00715488638531 1.5156910852306282, -58.127781886497644 1.5177820852325823, -58.26833588662855 1.570554085281728, -58.29722688665545 1.5827820852931183, -58.387290886739336 1.4778450851953835, -58.50229088684644 1.454673085173809, -58.46930888681571 1.3584730850842135, -58.46764588681417 1.3376360850648013, -58.47514588682115 1.3140270850428237, -58.51961788686258 1.2696180850014542, -58.57249988691183 1.2752820850067366, -58.69299088702404 1.2847180850155269, -58.7088908870
 3885 1.2629180849952206, -58.722917887051906 1.2323640849667612, -58.751945887078946 1.209582084945552, -58.810699887133666 1.1868730849244002, -58.8363998871576 1.1902820849275741, -58.86959088718851 1.2050000849412754, -58.89840888721535 1.2311090849655955, -58.89736388721437 1.261836084994215, -58.92486388723998 1.2993090850291082, -58.95750888727039 1.3144450850432037, -58.97278188728461 1.3197180850481232, -59.104726887407494 1.3447180850714062, -59.24396388753718 1.3865270851103446, -59.41445488769595 1.5633360852750116, -59.48361788776036 1.6505540853562337, -59.50750888778262 1.6875000853906386, -59.6054178878738 1.7194450854203893, -59.64576388791137 1.735136085435002, -59.675281887938866 1.7670820854647644, -59.653617887918685 1.8022180854974863, -59.632472887899 1.8422360855347506, -59.7263178879864 1.854164085545861, -59.74902688800755 1.8613910855525972, -59.75625488801428 1.908891085596835, -59.743617888002504 2.0727820857494663, -59.727017887987046 2.241245085906357, 
 -59.73549088799494 2.2847180859468494, -59.79319988804869 2.297782085959014, -59.84972688810133 2.330554085989533, -59.893263888141874 2.3641000860207697, -59.901672888149704 2.384445086039719, -59.88889088813781 2.407082086060811, -59.89278188814143 2.4544450861049114, -59.925563888171965 2.567782086210471, -59.9541728881986 2.616391086255746, -59.98847288823055 2.688191086322604, -59.98278188822525 2.860282086482883, -59.97889088822163 2.899164086519093, -59.96278188820662 3.025836086637071, -59.949717888194456 3.071945086680003, -59.90472688815255 3.2041640868031465, -59.869726888119956 3.2763910868704187, -59.85611788810728 3.2977820868903365, -59.83569988808827 3.322082086912971, -59.810835888065114 3.3595820869478956, -59.81444588806848 3.498745087077495, -59.831945888084775 3.5241640871011697, -59.786117888042085 3.620554087190939, -59.6655548879298 3.711391087275544, -59.56860888783952 3.8994450874506725, -59.58167288785168 3.9988910875432992, -59.70861788796991 4.1706910877
 0329, -59.722226887982586 4.191391087722579, -59.73110888799086 4.219164087748439, -59.73083588799061 4.293336087817522, -59.675281887938866 4.373336087892028, -59.67444588793809 4.385136087903007, -59.71722688797793 4.414164087930047, -59.79111788804674 4.456109087969111, -59.94180888818708 4.508054088017488, -59.96778188821128 4.501945088011794, -60.045281888283455 4.4940270880044295, -60.13069988836301 4.509718088019042, -60.14847288837956 4.52000008802861, -60.152226888383055 4.5733360880782925, -60.12389088835667 4.596664088100013, -60.09152688832653 4.600691088103758, -60.07209088830842 4.618473088120325, -60.02583588826535 4.707218088202978, -60.02056388826044 4.734445088228327, -60.00583588824672 4.8369450883237874, -59.983063888225516 5.022500088496599, -60.013890888254224 5.11000008857809, -60.080872888316605 5.161509088626062, -60.081945888317605 5.1730540886368175, -60.09833588833287 5.217218088677953, -60.114581888348 5.245691088704461, -60.202363888429744 5.27270908872
 9627, -60.24999988847412 5.2584730887163715, -60.268408888491265 5.235282088694774, -60.317845888537306 5.197636088659706, -60.54639088875015 5.191391088653901, -60.5738268887757 5.197154088659261, -60.597426888797685 5.211945088673033, -60.689445888883384 5.216109088676916, -60.730372888921494 5.2048000886663885, -60.748890888938746 5.222500088682864, -60.781108888968745 5.2583360887162485, -60.86028188904248 5.348054088799799, -61.11610888928074 5.63471808906678, -61.38972688953557 5.940000089351088, -61.37345488952042 5.960973089370626, -61.3378908894873 5.972782089381624, -61.32306388947349 5.992500089399982, -61.282817889436004 6.056336089459435, -61.270281889424325 6.086391089487435, -61.26278188941734 6.107782089507353, -61.160145889321754 6.1825000895769335, -61.1386178893017 6.29916408968559, -61.140008889303004 6.408609089787518, -61.19652688935564 6.533682089904005, -61.20472688936327 6.5779180899452, -61.174445889335075 6.658745090020474, -61.12510888928912 6.71477309007
 2654, -61.05472688922357 6.72889109008581, -61.03597288920611 6.719718090077265, -60.93965488911641 6.724582090081796, -60.89368188907359 6.765000090119429, -60.81611788900135 6.788336090141172, -60.71917288891106 6.759027090113875, -60.69778188889114 6.766664090120983, -60.676390888871225 6.790000090142712, -60.66499988886062 6.80750009015901, -60.63778188883526 6.834927090184564, -60.590835888791545 6.85083609019938, -60.526945888732044 6.87471809022162, -60.498890888705915 6.8877820902337845, -60.463617888673056 6.906664090251368, -60.40833588862158 6.947500090289395, -60.29125488851254 7.056600090391015, -60.28139088850335 7.090000090422109, -60.278054888500236 7.118745090448883, -60.309035888529095 7.143473090471915, -60.33667288855483 7.155282090482913, -60.36049088857702 7.176245090502434, -60.45611788866607 7.195836090520686, -60.50576388871231 7.17270909049914, -60.51499988872092 7.142982090471463, -60.5380638887424 7.124445090454188, -60.61749988881638 7.19444509051938, -6
 0.6308358888288 7.221945090544992, -60.63423588883197 7.254309090575134, -60.61861788881741 7.2840270906028195, -60.595354888795754 7.30610909062338, -60.59139088879206 7.336391090651588, -60.6099998888094 7.381945090694003, -60.66083588885674 7.447218090754802, -60.688263888882275 7.453473090760625, -60.714026888906275 7.500973090804862, -60.71666388890873 7.540000090841204, -60.691108888884926 7.567082090866435, -60.662090888857904 7.566527090865918, -60.58875488878961 7.639309090933693, -60.57458188877641 7.7152090910043825, -60.533335888737994 7.803891091086982, -60.51319988871924 7.818191091100289, -60.31972688853905 7.877500091155525, -60.25754588848115 7.921573091196578, -60.23722688846222 7.946391091219695, -60.12486388835757 8.028327091296006, -60.09360888832846 8.041245091308028, -60.03778188827647 8.037009091304085, -60.0111818882517 8.059236091324792, -59.98701788822919 8.1469820914065, -59.9797268882224 8.174864091432468, -59.94618188819116 8.199991091455871, -59.922226
 88816885 8.211109091466227, -59.88889088813781 8.218609091473212, -59.83285488808562 8.231527091485248, -59.80305488805786 8.2833360915335, -59.81209088806628 8.306109091554703, -59.88611788813522 8.40083609164293, -59.98111788822369 8.518327091752354, -59.99028188823223 8.535273091768133, -59.95819988820236 8.514027091748346, -59.92930888817544 8.48416409172053, -59.902572888150544 8.445764091684765, -59.88945488813833 8.421664091662322, -59.87250888812255 8.397218091639559, -59.85583588810702 8.379445091622998, -59.83167288808451 8.361664091606443, -59.78625488804222 8.340973091587173, -59.76472688802217 8.349027091594678, -59.77930888803574 8.380973091624426, -59.764208888021685 8.407364091649, -59.6683358879324 8.360000091604888, -59.6508358879161 8.349445091595058, -59.35583588764136 8.173891091431571, -59.15472688745406 8.05639109132214, -59.12930888743038 8.040000091306865, -59.109445887411894 8.018473091286822, -59.09347288739701 7.987009091257519, -59.07583588738059 7.96916
 40912408985, -58.94389088725771 7.851109091130951, -58.80694588713017 7.730554091018675, -58.76889088709473 7.679718090971335, -58.751399887078435 7.6361090909307165, -58.71736388704673 7.59437309089185, -58.6419458869765 7.569445090868626, -58.48528188683059 7.368609090681588, -58.46854588681501 7.337573090652683, -58.465563886812234 7.135836090464807, -58.46666388681325 7.115000090445392, -58.4815268868271 7.009927090347546, -58.497508886841985 6.989445090328459, -58.51074588685431 6.980254090319903, -58.53709088687884 6.96069109030168, -58.55569988689618 6.937500090280082, -58.56305488690302 6.916391090260433, -58.59403588693188 6.801809090153711, -58.598608886936134 6.774309090128099, -58.60917288694597 6.656664090018538, -58.619090886955206 6.489491089862852, -58.637090886971976 6.421945089799934, -58.600726886938105 6.413945089792492, -58.59333588693123 6.420000089798123, -58.58132688692004 6.464973089840015, -58.574026886913245 6.509864089881816, -58.57278188691208 6.53250008
 9902896, -58.576117886915185 6.562218089930582, -58.57319988691248 6.602636089968215, -58.54972688689061 6.6782640900386525, -58.47944588682516 6.794445090146851, -58.45695488680421 6.827782090177905, -58.41972688676954 6.870282090217486, -58.39833588674962 6.879309090225888, -58.3165998866735 6.894236090239801, -58.25569988661678 6.876109090222911, -58.208617886572924 6.843054090192126, -58.1726178865394 6.812218090163412)))	
+PCN	Pitcairn Is.	MULTIPOLYGON (((-128.33221795188064 -24.32726393883749, -128.3269359518757 -24.326117938836433, -128.31042695186034 -24.325835938836164, -128.30168195185217 -24.334445938844183, -128.29388195184492 -24.352499938861, -128.2866819518382 -24.38639093889256, -128.2861179518377 -24.40166393890678, -128.29111795184235 -24.411390938915844, -128.30156395185207 -24.411672938916112, -128.3130819518628 -24.404717938909627, -128.3278179518765 -24.390554938896443, -128.34308195189075 -24.367217938874703, -128.34695495189436 -24.353617938862044, -128.34600895189345 -24.33819993884768, -128.336945951885 -24.329163938839258, -128.33221795188064 -24.32726393883749)), ((-130.08139095350967 -25.082226939540604, -130.09068195351833 -25.080699939539187, -130.10262695352944 -25.074445939533362, -130.1050549535317 -25.061463939521275, -130.0905639535182 -25.055699939515904, -130.06384595349334 -25.06826393952761, -130.0769359535055 -25.079726939538276, -130.08139095350967 -25.082226939540
 604)))	
+SGS	South Georgia & the South Sandwich Is.	MULTIPOLYGON (((-36.99139086681285 -54.35056396679887, -36.99972686682062 -54.348617966797065, -37.05944586687622 -54.329999966779724, -37.08375486689886 -54.31881796676931, -37.08222686689746 -54.29389096674609, -37.146117866956956 -54.26139096671582, -37.241808867046075 -54.247363966702764, -37.266254867068824 -54.25875496671337, -37.365835867161564 -54.271944966725655, -37.408335867201146 -54.26493596671913, -37.403472867196626 -54.18500896664469, -37.367226867162884 -54.17556396663589, -37.33555486713337 -54.17361796663408, -37.25208186705564 -54.15229096661422, -37.48986386727708 -54.12944496659294, -37.56118186734349 -54.14607296660843, -37.52944586731394 -54.16250896662373, -37.56222686734449 -54.170563966631235, -37.58944586736982 -54.17528196663563, -37.650417867426626 -54.18167296664158, -37.68528186745908 -54.17528196663563, -37.71917286749064 -54.140563966603295, -37.71757286748917 -54.09416396656008, -37.676390867450806 -54.074
 717966541975, -37.653826867429785 -54.072154966539586, -37.61979086739808 -54.04611796651534, -37.69221786746553 -54.034999966504984, -37.89778186765699 -54.04722696651637, -38.02257286777322 -54.054726966523354, -38.0237548677743 -54.00743596647931, -37.93361786769037 -53.99222696646515, -37.91028186766863 -53.98972696646282, -37.74721786751675 -53.99499996646773, -37.49499986728188 -54.01056396648222, -37.46139986725058 -54.036117966506026, -37.37389086716908 -54.04805496651714, -37.27028186707258 -54.050281966519215, -37.162781866972466 -54.03139096650162, -37.02778186684674 -54.05611796652465, -36.80944586664339 -54.08833596655465, -36.65639086650086 -54.107781966572766, -36.62375486647045 -54.120699966584795, -36.584163866433585 -54.20889096666693, -36.66757286651128 -54.24409996669972, -36.662645866506665 -54.27514496672863, -36.62667286647317 -54.28361796673653, -36.604026866452074 -54.26611796672023, -36.55680886640812 -54.245563966701084, -36.517081866371115 -54.23445496669
 074, -36.47889086633555 -54.23889096669487, -36.47263586632971 -54.26513596671931, -36.456117866314344 -54.3275089667774, -36.36999986623414 -54.35499996680301, -36.33667286620309 -54.35229096680048, -36.362090866226765 -54.293054966745316, -36.38368186624686 -54.27826396673154, -36.394799866257216 -54.24917296670444, -36.290208866159816 -54.26583596671996, -36.257226866129116 -54.28666396673936, -36.22917286610297 -54.33778196678697, -36.25610886612807 -54.36847296681555, -36.15999986603856 -54.44472696688657, -36.09680886597971 -54.54959096698423, -36.06389086594905 -54.57153596700467, -35.979726865870674 -54.57889096701152, -35.93471786582873 -54.62306396705266, -35.933890865827976 -54.700563967124836, -35.917363865812575 -54.715144967138414, -35.85667286575605 -54.74305496716441, -35.8280548657294 -54.75083596717165, -35.79396386569766 -54.76020896718038, -35.82778186572915 -54.79278196721072, -35.914172865809604 -54.81347296722999, -35.95944586585176 -54.81110896722779, -35.979
 726865870674 -54.80917296722598, -36.0006998658902 -54.80083596721822, -36.02528186591309 -54.787781967206065, -36.04583586593222 -54.77889096719778, -36.074717865959116 -54.768054967187695, -36.095308865978296 -54.76993596718945, -36.02111786590922 -54.81721796723348, -35.98972686587999 -54.8305549672459, -35.96430886585631 -54.83167296724694, -35.923617865818414 -54.8504179672644, -35.96139986585359 -54.86944496728212, -36.07639086596069 -54.890281967301526, -36.10472686598709 -54.88923596730055, -36.131463866011984 -54.8688909672816, -36.19917286607503 -54.80917296722598, -36.29749986616662 -54.733335967155355, -36.30250886617128 -54.71305496713647, -36.31721786618496 -54.686954967112165, -36.46860886632598 -54.52889096696495, -36.525908866379325 -54.497081966935326, -36.582226866431796 -54.49986396693792, -36.65555486650007 -54.491108966929765, -36.736945866575866 -54.47028196691037, -36.809799866643715 -54.44541796688721, -36.801108866635644 -54.41118196685533, -36.864935866695
 08 -54.34652696679511, -36.92528186675128 -54.337917966787096, -36.99139086681285 -54.35056396679887)), ((-26.248890856808117 -58.49860897066204, -26.264163856822336 -58.48806397065222, -26.290972856847304 -58.47861797064343, -26.37167285692246 -58.46305497062893, -26.389726856939262 -58.45972697062583, -26.406526856954912 -58.45944497062557, -26.41999985696748 -58.45667297062299, -26.458199857003052 -58.430699970598795, -26.433608856980157 -58.38916397056011, -26.416663856964362 -58.38360897055494, -26.403890856952472 -58.382508970553914, -26.39305485694237 -58.38278197055417, -26.317781856872273 -58.38639097055753, -26.29152685684781 -58.38806397055909, -26.265699856823773 -58.39417297056478, -26.246672856806043 -58.404444970574346, -26.241390856801132 -58.47167297063696, -26.243617856803212 -58.49472697065843, -26.248890856808117 -58.49860897066204)))	
+SHN	St. Helena	MULTIPOLYGON (((-5.712972837682543 -15.992863931075476, -5.729163837697627 -16.005490931087238, -5.768472837734237 -16.021945931102564, -5.787217837751683 -16.009163931090654, -5.792781837756877 -15.991108931073839, -5.768890837734631 -15.94736393103311, -5.748608837715722 -15.929163931016149, -5.728890837697378 -15.91389093100193, -5.716390837685736 -15.905281930993908, -5.699517837670015 -15.903754930992491, -5.671390837643827 -15.90944593099779, -5.645281837619507 -15.93999993102625, -5.646390837620544 -15.958335931043322, -5.660417837633588 -15.985690931068802, -5.700417837670841 -16.003754931085624, -5.7105548376802915 -15.996390931078764, -5.712972837682543 -15.992863931075476)))	
+SUR	Suriname	MULTIPOLYGON (((-55.12796388370384 5.82217308924136, -55.10444588368195 5.839445089257438, -55.01610888359967 5.850418089267663, -54.946672883535 5.846109089263649, -54.86444588345843 5.855136089272051, -54.88729088347971 5.879545089294794, -54.9688908835557 5.873336089289012, -54.99534588358034 5.865764089281953, -55.10625488368363 5.904582089318112, -55.14472688371946 5.934164089345657, -55.15944588373317 5.9634000893728825, -55.047363883628776 6.001809089408653, -54.97028188355699 5.988054089395845, -54.87694588347007 5.985282089393266, -54.77944588337927 5.982500089390669, -54.70994588331453 5.962436089371991, -54.639163883248614 5.954718089364803, -54.34445488297415 5.906945089320303, -54.29861788293145 5.898054089312026, -54.204726882844014 5.879718089294954, -54.17860888281969 5.871391089287201, -54.02556388267716 5.818609089238038, -53.990208882644225 5.746945089171291, -54.00652688265943 5.721245089147359, -54.04646388269663 5.653536089084298, -54.0515358827013
 44 5.590209089025322, -54.05180888270159 5.527082088966537, -54.067781882716474 5.491527088933424, -54.095835882742605 5.458054088902244, -54.120490882765566 5.431600088877616, -54.142226882785806 5.39527308884378, -54.16668188280859 5.347400088799191, -54.17069988281233 5.342218088794368, -54.19333588283341 5.315273088769274, -54.244999882881515 5.2850000887410715, -54.28348188291736 5.254864088713006, -54.317326882948876 5.208627088669942, -54.347781882977245 5.148336088613803, -54.37889088300622 5.112218088580164, -54.4086178830339 5.08055408855067, -54.448063883070645 5.0241640884981535, -54.45528188307736 5.004027088479404, -54.47610888309676 4.867773088352507, -54.47778188309832 4.7541640882466965, -54.47333588309418 4.734582088228464, -54.439726883062875 4.669164088167534, -54.42209088304645 4.601527088104547, -54.42326388304754 4.5643090880698765, -54.442917883065846 4.530691088038566, -54.45180888307412 4.512364088021499, -54.44023588306335 4.410645087926767, -54.4313998830
 5512 4.369164087888137, -54.39389088302019 4.237218087765257, -54.365008882993294 4.163609087696699, -54.36076388298933 4.042427087583846, -54.29028188292369 3.9338910874827633, -54.18868188282907 3.808745087366205, -54.14639088278969 3.7977820873559978, -54.12764588277223 3.788609087347453, -53.99778188265128 3.626945087196887, -53.986117882640414 3.601527087173224, -54.00110888265438 3.4483360870305546, -54.07110888271957 3.3127820869043063, -54.09445488274132 3.2950000868877396, -54.11875488276395 3.27778208687171, -54.187226882827716 3.1948640867944818, -54.206117882845305 3.14528208674831, -54.2084728828475 3.124309086728772, -54.19574588283565 3.096200086702595, -54.176117882817366 3.043891086653886, -54.16083588280314 2.9536090865697986, -54.19528188283522 2.796245086423241, -54.20459088284389 2.775000086403452, -54.222917882860955 2.755554086385345, -54.25695488289266 2.7188910863512064, -54.31513588294685 2.629164086267636, -54.32180888295305 2.5984730862390535, -54.3519548
 8298113 2.5238910861695985, -54.365908882994134 2.49660008614417, -54.420563883045034 2.434718086086548, -54.499172883118234 2.3650000860216096, -54.60378188321566 2.3291910859882705, -54.63472688324448 2.3200730859797716, -54.687499883293626 2.3255540859848765, -54.709099883313755 2.3879180860429585, -54.69020888329615 2.3995090860537545, -54.68986388329583 2.4516000861022604, -54.75312688335475 2.4706270861199897, -54.77538188337547 2.456845086107151, -54.80375488340191 2.4380540860896502, -54.84749988344264 2.436109086087839, -54.86916388346282 2.44194508609327, -54.96944588355622 2.550554086194424, -55.091945883670306 2.53471808617968, -55.11194588368893 2.527218086172695, -55.50333588405344 2.438891086090436, -55.71368188424934 2.4001360860543315, -55.86409988438943 2.471454086120758, -55.88639088441019 2.49708208614463, -55.91750888443917 2.5206910861666074, -55.94180888446179 2.5305540861757976, -55.96278188448133 2.533054086178126, -55.98249988449969 2.522082086167913, -56.0
 0041788451638 2.4508360861015603, -56.08444588459464 2.3575000860146247, -56.11583588462388 2.2491640859137334, -56.03639088454989 2.208891085876232, -55.90396388442656 2.0477820857261833, -55.90173588442448 1.90104508558953, -55.94264588446258 1.856109085547672, -55.96583588448418 1.8452820855375904, -55.99597288451224 1.8376360855304625, -56.0274998845416 1.8362450855291712, -56.06806388457939 1.8455540855378416, -56.11333588462155 1.863609085554657, -56.201663884703805 1.8916640855807856, -56.326954884820495 1.9252820856120962, -56.426390884913104 1.9341640856203668, -56.470635884954305 1.944500085629997, -56.47736388496057 1.9620820856463723, -56.55597288503378 2.020554085700823, -56.579726885055905 2.0273640857071626, -56.64611788511773 2.028054085707808, -56.68209088515124 2.0267640857066027, -56.77499988523776 2.1408360858128503, -56.810563885270895 2.1933360858617448, -56.853890885311245 2.283336085945564, -56.917781885370744 2.428054086080337, -56.94472688539584 2.483054086
 1315598, -57.11688188555617 2.757191086386868, -57.20083588563436 2.8228450864480124, -57.213335885646 2.878891086500218, -57.207217885640304 3.004236086616956, -57.210281885643155 3.035836086646384, -57.22028188565247 3.0650000866735354, -57.255008885684816 3.103609086709497, -57.28361788571145 3.140973086744296, -57.28999988571739 3.181391086781943, -57.29249988571972 3.2673640868620026, -57.28354588571139 3.318682086909803, -57.304172885730594 3.380418086967296, -57.42945488584728 3.3472180869363797, -57.53055488594143 3.3463910869356113, -57.61041788601581 3.3600000869482756, -57.642226886045435 3.3563910869449245, -57.65778188605992 3.4138910869984755, -57.63840888604189 3.457009087038628, -57.65444588605682 3.518609087095996, -57.66861788607001 3.5383360871143736, -57.740835886137276 3.613745087184597, -57.76056388615565 3.624164087194302, -57.8166728862079 3.6502820872186277, -57.83848188622821 3.6693090872363427, -57.84597288623519 3.692636087258066, -57.84930888623829 3.738
 8910873011554, -57.85945488624775 3.771664087331672, -57.86749988625523 3.7908360873495326, -57.876672886263776 3.8091640873665966, -57.93194588631526 3.8906910874425193, -57.95041788633246 3.913191087463474, -57.96889088634967 3.9291640874783553, -58.01639988639391 3.9636090875104344, -58.04375488641939 4.001527087545753, -58.071399886445136 4.15569108768932, -58.0618088864362 4.180136087712086, -58.039172886415116 4.211391087741205, -58.00278188638123 4.239718087767585, -57.954517886336276 4.288336087812866, -57.9491728863313 4.317218087839763, -57.94861788633078 4.349164087869511, -57.902781886288096 4.503609088013349, -57.8469458862361 4.6288910881300325, -57.84035488622996 4.669027088167411, -57.88778188627413 4.7683360882599, -57.909026886293915 4.777782088268694, -57.92486388630866 4.796391088286029, -57.92333588630724 4.821945088309818, -57.901390886286805 4.855554088341123, -57.84111788623066 4.927782088408392, -57.67625488607713 5.011527088486389, -57.63417288603793 5.0002
 82088475913, -57.61098188601633 4.992218088468405, -57.4736178858884 4.989164088465557, -57.41709088583576 4.989445088465814, -57.32722688575207 5.026109088499965, -57.321945885747155 5.075973088546405, -57.292363885719595 5.166245088630475, -57.27209088570072 5.177082088640574, -57.25055488568066 5.172636088636423, -57.231526885662944 5.146454088612046, -57.19104588562524 5.172009088635846, -57.20750888564058 5.223609088683901, -57.23520888566637 5.259945088717743, -57.26222688569153 5.22166408868209, -57.282781885710676 5.226391088686498, -57.32444588574948 5.303609088758407, -57.2994458857262 5.359718088810666, -57.27159988570027 5.384864088834078, -57.24850888567876 5.486109088928373, -57.18430888561896 5.51889108895891, -57.16742688560325 5.541391088979864, -57.13889088557667 5.671664089101185, -57.13597288557395 5.6925000891205855, -57.13249988557071 5.767500089190435, -57.06791788551057 5.941736089352716, -56.992499885440324 5.989445089397137, -56.96444588541421 5.99708208940
 4259, -56.70056388516845 5.965554089374891, -56.60361788507815 5.940500089351559, -56.55445488503237 5.948336089358861, -56.48000888496304 5.945418089356139, -56.26056388475867 5.889164089303748, -56.01778188453255 5.818336089237789, -55.910554884432685 5.778364089200565, -55.89444588441769 5.730273089155773, -55.89860888442156 5.67444508910377, -55.89955488442244 5.671909089101419, -55.885972884409796 5.683054089111792, -55.87778188440217 5.717636089143994, -55.89110888441458 5.775836089198208, -55.922717884444026 5.876073089291552, -55.85583588438173 5.948891089359378, -55.8277818843556 5.958336089368174, -55.76806388429999 5.967218089376445, -55.62028188416235 5.974445089383167, -55.548617884095606 5.977782089386281, -55.4127818839691 5.964164089373597, -55.37722688393599 5.960000089369714, -55.33902688390042 5.950136089360527, -55.266672883833024 5.923609089335827, -55.25389088382113 5.915836089328593, -55.230563883799405 5.898054089312026, -55.174999883747645 5.9072180893205655
 , -55.11555488369228 5.876982089292397, -55.11402688369087 5.8400000892579556, -55.12796388370384 5.82217308924136)))	
+TTO	Trinidad & Tobago	MULTIPOLYGON (((-61.0794458892466 10.824164093899824, -61.075563889242986 10.826109093901636, -61.02541788919628 10.840273093914831, -60.92833588910587 10.838609093913277, -60.90923588908808 10.827009093902475, -60.92305488910094 10.797218093874733, -60.94445488912088 10.762218093842137, -60.96305488913819 10.739164093820662, -60.98722688916071 10.714718093797899, -61.01791788918929 10.698473093782766, -61.03417288920443 10.678191093763871, -61.0454178892149 10.489164093587831, -61.02097288919214 10.392218093497547, -60.99860888917131 10.351600093459723, -61.00444588917675 10.149582093271576, -61.083617889250476 10.102773093227981, -61.10722688927247 10.091945093217888, -61.15222688931438 10.075554093202626, -61.202917889361586 10.069164093196676, -61.405281889550054 10.066945093194605, -61.526672889663104 10.06833609319591, -61.67708188980319 10.076109093203144, -61.7055548898297 10.07889109320574, -61.80611788992336 10.0811090932078, -61.83361788994897 10.071
 391093198756, -61.858335889972 10.06221809319021, -61.88028188999243 10.042500093171839, -61.90951789001966 10.040345093169833, -61.92159989003092 10.064864093192668, -61.817781889934224 10.127773093251264, -61.6477818897759 10.19721809331594, -61.50409988964209 10.237636093353572, -61.46139088960231 10.274309093387728, -61.4537548895952 10.294164093406224, -61.46305488960385 10.571245093664274, -61.47361788961369 10.597773093688986, -61.49583588963439 10.63527309372391, -61.54625488968135 10.669445093755726, -61.60125488973257 10.684582093769833, -61.62291788975274 10.678891093764534, -61.65778188978521 10.680064093765623, -61.662017889789155 10.708400093792008, -61.60278188973399 10.74221809382351, -61.47999988961965 10.75055409383127, -61.42854588957172 10.753954093834437, -61.384590889530784 10.779718093858435, -61.24249988939846 10.790273093868265, -61.19639988935552 10.789445093867485, -61.17180888933261 10.79910009387649, -61.1433358893061 10.815273093891548, -61.079445889246
 6 10.824164093899824)), ((-60.79749988898402 11.14166409419552, -60.84493588902819 11.157218094210009, -60.847226889030324 11.176945094228373, -60.756672888945985 11.242218094289171, -60.70083588889399 11.275836094320482, -60.65778188885389 11.301391094344282, -60.5295908887345 11.345554094385406, -60.52083588872635 11.330554094371436, -60.52472688872997 11.273891094318671, -60.53180888873656 11.259027094304827, -60.64180888883901 11.202218094251919, -60.66889088886424 11.198891094248822, -60.68778188888183 11.201945094251656, -60.73875488892931 11.182636094233672, -60.758199888947416 11.171245094223067, -60.79749988898402 11.14166409419552)))	
+VEN	Venezuela	MULTIPOLYGON (((-66.3102908941182 10.626018093715288, -66.28309089409287 10.64465409373264, -66.22987289404331 10.640554093728824, -66.0816818939053 10.576664093669322, -66.11972689394072 10.500000093597919, -65.95861789379067 10.356664093464431, -65.93573589376938 10.298300093410077, -65.81445489365642 10.228336093344922, -65.0813908929737 10.060554093188657, -64.76445489267853 10.097218093222807, -64.73279089264904 10.113473093237943, -64.71473589263222 10.179718093299641, -64.62597289254956 10.246664093361986, -64.57972689250649 10.25860909337311, -64.48250889241595 10.238054093353966, -64.37854589231912 10.301109093412691, -64.39584589233523 10.337218093446324, -64.36805489230936 10.384164093490043, -64.18205489213612 10.456545093557452, -63.80944589178911 10.442218093544113, -63.697499891684856 10.485554093584469, -63.83889089181653 10.551664093646039, -63.966390891935276 10.578473093671008, -64.14868189210505 10.570000093663111, -64.20098189215375 10.550273093644
 748, -64.23569089218608 10.514373093611312, -64.29959089224559 10.626527093715765, -64.264590892213 10.657773093744865, -64.1468088921033 10.617918093707743, -63.84278189182015 10.645827093733743, -63.534445891533 10.627082093716282, -63.25306389127094 10.678336093764017, -63.17222689119565 10.719718093802555, -62.99375489102944 10.716391093799459, -62.90667289094833 10.695836093780315, -62.697499890753534 10.747773093828684, -62.19250889028322 10.694164093778753, -61.87959088999179 10.728327093810577, -61.88333588999528 10.694718093779272, -61.9530548900602 10.648891093736594, -62.0855548901836 10.627218093716408, -62.249172890335984 10.627009093716211, -62.331108890412295 10.531664093627413, -62.527226890594946 10.540418093635566, -62.64972689070903 10.568191093661426, -62.91229989095358 10.528745093624693, -63.0042358910392 10.45298209355414, -62.87250889091652 10.524445093620685, -62.83860889088494 10.511664093608786, -62.8377088908841 10.397291093502275, -62.87180889091586 10.3
 89864093495348, -62.934308890974066 10.418473093521996, -63.00368189103868 10.394164093499356, -62.998408891033776 10.271600093385217, -62.93417289097394 10.279309093392385, -62.946663890985576 10.400900093505626, -62.8847358909279 10.374391093480938, -62.790090890839764 10.401336093506032, -62.73639089078975 10.37381809348041, -62.66639089072456 10.280554093393548, -62.630626890691246 10.107154093232054, -62.669308890727265 10.075000093202107, -62.82917289087615 10.053609093182189, -62.95694589099516 10.104309093229404, -63.015972891050126 10.095691093221376, -62.983854891020215 10.068918093196444, -62.91056389095196 10.076945093203918, -62.805281890853905 10.00860909314028, -62.66278189072119 10.059164093187363, -62.6157638906774 10.093191093219048, -62.605417890667766 10.127218093250747, -62.62333589068446 10.188609093307917, -62.60944589067152 10.223954093340836, -62.577499890641775 10.22513609334193, -62.48889089055925 10.153054093274804, -62.42694589050156 9.979164093112857, -
 62.31518189039747 9.752845092902078, -62.32264589040442 9.712082092864122, -62.277226890362115 9.747500092897099, -62.23680889032447 9.855000092997216, -62.25049089033722 9.967009093101538, -62.205699890295506 9.914027093052198, -62.165345890257925 9.715282092867099, -62.19610889028657 9.641664092798536, -62.17076389026296 9.657427092813222, -62.130008890225014 9.752109092901392, -62.133499890228265 9.82644509297063, -62.02743589012948 9.866418093007852, -62.11222689020845 9.929445093066548, -62.20153589029162 9.938336093074838, -62.231735890319754 9.964718093099407, -62.2092358902988 10.011036093142536, -62.17889089027054 10.014718093145973, -62.05389089015412 9.977500093111303, -61.96222689006875 9.91194509305025, -61.81167288992853 9.757891092906789, -61.73583588985791 9.600936092760605, -61.73333588985558 9.698745092851695, -61.80091788991852 9.812273092957426, -61.79278188991094 9.83110909297497, -61.77194588989154 9.830273092974195, -61.59770888972926 9.782773092929958, -61.57
 458188970773 9.800691092946636, -61.64805488977615 9.897364093036671, -61.619863889749894 9.90528209304405, -61.48056388962017 9.823609092967985, -61.407226889551865 9.704718092857263, -61.255981889411004 9.588891092749392, -61.04333588921297 9.575554092736965, -60.98757288916103 9.551809092714848, -60.85361788903627 9.444445092614856, -60.79819988898467 9.379309092554195, -60.78361788897108 9.305000092484988, -60.81889088900394 9.26889109245137, -60.987781889161226 9.188609092376595, -61.08459088925139 9.097500092291739, -61.098126889263995 9.043954092241876, -60.972708889147185 9.175209092364113, -60.9504908891265 9.175136092364042, -61.031535889201976 9.032082092230823, -61.09778188926367 8.963336092166799, -61.183335889343354 8.727782091947418, -61.210208889368374 8.595136091823875, -61.25090888940629 8.581454091811139, -61.436672889579285 8.60166409182996, -61.51972688965664 8.590418091819487, -61.59541788972713 8.616945091844187, -61.59889088973037 8.555000091786496, -61.39389
 088953945 8.474445091711473, -61.32917288947917 8.430827091670864, -61.16847288932951 8.495764091731331, -61.0775088892448 8.493054091728808, -61.0861088892528 8.421391091662073, -61.0735458892411 8.402918091644864, -61.017781889189166 8.46916409170656, -60.98125488915515 8.564445091795292, -60.71583588890796 8.604582091832683, -60.62716388882538 8.551182091782948, -60.46583588867513 8.5282640917616, -60.40875488862197 8.621600091848535, -60.23611788846118 8.627500091854017, -59.99028188823223 8.535273091768133, -59.80305488805786 8.2833360915335, -59.83285488808562 8.231527091485248, -59.92222688816885 8.211109091466227, -59.9797268882224 8.174864091432468, -60.0111818882517 8.059236091324792, -60.03778188827647 8.037009091304085, -60.12486388835757 8.028327091296006, -60.31972688853905 7.877500091155525, -60.533335888737994 7.803891091086982, -60.58875488878961 7.639309090933693, -60.662090888857904 7.566527090865918, -60.691108888884926 7.567082090866435, -60.71666388890873 7.540
 000090841204, -60.688263888882275 7.453473090760625, -60.66083588885674 7.447218090754802, -60.59139088879206 7.336391090651588, -60.63423588883197 7.254309090575134, -60.61749988881638 7.19444509051938, -60.5380638887424 7.124445090454188, -60.50576388871231 7.17270909049914, -60.45611788866607 7.195836090520686, -60.36049088857702 7.176245090502434, -60.278054888500236 7.118745090448883, -60.29125488851254 7.056600090391015, -60.463617888673056 6.906664090251368, -60.63778188883526 6.834927090184564, -60.69778188889114 6.766664090120983, -60.81611788900135 6.788336090141172, -60.89368188907359 6.765000090119429, -60.93965488911641 6.724582090081796, -61.12510888928912 6.714773090072654, -61.174445889335075 6.658745090020474, -61.20472688936327 6.5779180899452, -61.140008889303004 6.408609089787518, -61.160145889321754 6.1825000895769335, -61.26278188941734 6.107782089507353, -61.3378908894873 5.972782089381624, -61.38972688953557 5.940000089351088, -60.748890888938746 5.2225000886
 82864, -60.65520888885149 5.181318088644517, -60.5785458887801 4.952636088431532, -60.649308888845994 4.842918088329355, -60.7202818889121 4.776109088267134, -60.88652688906693 4.709718088205307, -60.92986388910728 4.591873088095554, -60.987090889160584 4.519309088027967, -61.163890889325245 4.494309088004684, -61.313608889464675 4.506664088016194, -61.50736388964512 4.390554087908058, -61.51069988964824 4.3034730878269585, -61.55438188968891 4.248818087776058, -61.69917288982376 4.259027087785569, -61.76291788988313 4.2430540877706875, -61.8488908899632 4.160554087693853, -61.914863890024634 4.146945087681175, -61.987499890092295 4.169582087702267, -62.05319989015348 4.150136087684146, -62.14264589023678 4.093191087631112, -62.440135890513844 4.1826730877144485, -62.539935890606785 4.112291087648913, -62.6041638906666 4.041945087583386, -62.72833589078225 4.038609087580284, -62.75778189080967 4.020273087563211, -62.78041789083075 3.9087450874593372, -62.72749989078147 3.73110908729
 3902, -62.73402689078755 3.6765270872430733, -62.78125489083153 3.604309087175807, -62.878054890921675 3.5601360871346657, -62.99056389102647 3.604309087175807, -63.24229089126091 3.8981270874494527, -63.335554891347755 3.9580540875052606, -63.419099891425574 3.967082087513674, -63.45819989146199 3.866454087419953, -63.491526891493024 3.857782087411877, -63.530135891528985 3.8672180874206674, -63.61417289160724 3.9448640874929737, -63.64445489163545 3.9486090874964646, -63.84806389182508 3.9588910875060463, -63.95625489192584 3.891245087443039, -64.01779089198314 3.886109087438257, -64.05270889201566 3.9084360874590516, -64.12639089208429 4.1095820876463875, -64.24889989219838 4.148054087682212, -64.33555489227909 4.154164087687903, -64.59201789251793 4.1277730876633285, -64.67472689259496 4.257500087784138, -64.74626389266159 4.287218087811823, -64.78167289269456 4.2863910878110545, -64.80000889271165 4.265073087791194, -64.80195489271345 4.210836087740688, -64.71313589263073 4.144
 027087678467, -64.62668189255022 3.9655540875122455, -64.44632689238225 3.7870090873459645, -64.30806389225349 3.7183360872820117, -64.19021789214374 3.5896540871621596, -64.18471789213861 3.5277820871045407, -64.23569989218609 3.432782087016065, -64.20529089215776 3.1933360867930674, -64.15889089211456 3.060836086669667, -63.987854891955266 2.718609086350938, -63.995417891962305 2.623054086261945, -64.04646389200985 2.509445086156134, -64.03403589199827 2.4713180861206325, -63.821808891800615 2.4258360860782773, -63.58167289157697 2.434718086086548, -63.36541789137557 2.4200000860728323, -63.346390891357856 2.4061090860598995, -63.360763891371235 2.259100085922981, -63.39944589140727 2.149509085820924, -63.60709089160065 2.105973085780377, -63.739172891723655 2.003054085684525, -63.825008891803606 1.9777820856609907, -63.93652689190746 1.9768090856600793, -64.00236389196877 1.9498640856349851, -64.0518908920149 1.8904820855796913, -64.07799089203921 1.6354180853421383, -64.11320889
 207201 1.5829180852932438, -64.26445489221287 1.4761090851937695, -64.3943088923338 1.5118090852270143, -64.52972689245992 1.4336090851541883, -64.59529089252098 1.3300000850576907, -64.69750889261618 1.262782084995095, -64.7572358926718 1.2443090849778855, -64.8190728927294 1.2794820850106419, -65.00695489290437 1.1658360849048108, -65.1362638930248 1.118473084860696, -65.16584589305235 0.9694450847218974, -65.19250889307719 0.9266640846820593, -65.32167289319747 0.9119450846683463, -65.38584589325724 0.8370820845986344, -65.40028189327069 0.7539540845212116, -65.43610889330405 0.6952820844665695, -65.52166389338373 0.6491640844236173, -65.55861789341814 0.6661090844393982, -65.58257289344046 0.7297910844987143, -65.5111178933739 0.8388910846003199, -65.510135893373 0.9019450846590331, -65.55944589341892 0.9711090847234516, -65.59542689345243 0.9904180847414352, -65.73529089358269 0.9827820847343247, -65.87083589370893 0.9083360846649953, -65.97959089381021 0.798745084562924, -66.1
 2709089394758 0.7341640845027797, -66.31472689412233 0.7513910845188292, -66.87045489463989 1.2209270849561165, -66.89168189465967 1.2513910849844905, -66.92639089469199 1.4597180851785083, -66.99167289475278 1.6958360853984118, -67.21153589495755 2.2436090859085596, -67.21694589496259 2.275282085938059, -67.17418189492277 2.336527085995101, -67.19250889493983 2.392500086047221, -67.32520889506341 2.474027086123158, -67.48556389521276 2.653336086290153, -67.53528189525906 2.680000086314976, -67.60583589532477 2.7933360864205383, -67.76583589547378 2.832500086457003, -67.82361789552759 2.8270820864519663, -67.8330728955364 2.8766640864981383, -67.4355638951662 3.253891086849464, -67.34639989508315 3.313682086905146, -67.29285489503329 3.3960450869818573, -67.30659989504609 3.4527820870346915, -67.38111789511548 3.4859730870656023, -67.43709089516761 3.648336087216819, -67.49104589521787 3.7243090872875655, -67.59987289531922 3.740691087302821, -67.63514589535207 3.797636087355855, -6
 7.7085088954204 4.0469540875880625, -67.78667289549318 4.166527087699421, -67.80625489551143 4.231809087760212, -67.78479989549145 4.334754087856098, -67.85875489556032 4.561245088067025, -67.80158189550707 4.973236088450719, -67.8486358955509 5.306518088761123, -67.80389089550923 5.38326408883259, -67.64944589536539 5.478336088921139, -67.61590889533416 5.5481910889861865, -67.65167289536745 5.683191089111915, -67.61999989533797 5.792218089213463, -67.4139818951461 5.995536089402819, -67.49252689521924 6.121409089520043, -67.48924589521619 6.153745089550156, -67.45445489518379 6.1930540895867665, -67.49626389522273 6.2054180895982824, -67.56361789528545 6.262500089651439, -67.6352818953522 6.28513608967252, -67.83118189553464 6.30757308969342, -67.91812689561561 6.238191089628799, -67.99792689568993 6.207218089599962, -68.15515489583636 6.222773089614449, -68.31945489598938 6.1680540895634834, -68.45389989611459 6.190554089584438, -68.63840889628644 6.135482089533156, -68.827790896
 4628 6.186391089580567, -69.05639989667571 6.2161090896082385, -69.19520889680498 6.100418089500494, -69.24174589684833 6.084100089485304, -69.2707088968753 6.090973089491698, -69.31611789691759 6.148609089545374, -69.42945489702315 6.1186090895174345, -70.11917289766549 6.975836090315795, -70.2218178977611 6.9740270903141095, -70.294726897829 6.9384730902809935, -70.55250889806908 7.058336090392629, -70.7197268982248 7.098054090429613, -70.88653589838016 7.07507309040821, -71.02500889850913 6.984445090323803, -71.18126389865465 6.9634730903042765, -71.33250889879551 7.022364090359119, -71.46389989891787 7.023336090360033, -71.60139089904592 7.057918090392235, -71.70195489913958 7.046391090381505, -71.83278189926142 6.987218090326394, -71.99236389941004 7.016245090353422, -72.06609989947871 7.06241809039642, -72.13292689954095 7.173336090499731, -72.16459989957045 7.262500090582762, -72.15472689956125 7.325282090641238, -72.26250889966164 7.389445090700988, -72.40167289979124 7.4072
 18090717549, -72.47244589985715 7.497982090802083, -72.45972689984531 7.920554091195626, -72.38938189977979 8.047564091313916, -72.3286178997232 8.061809091327177, -72.3366728997307 8.151945091411122, -72.3881998997787 8.369445091613684, -72.66446390003598 8.641109091866696, -72.77972690014333 9.080273092275704, -72.8855639002419 9.11916409231192, -72.96250890031357 9.178054092366764, -72.9811269003309 9.260827092443861, -73.0099999003578 9.302009092482209, -73.22195490055519 9.171109092360297, -73.37806390070058 9.171391092360565, -73.36389090068738 9.225827092411265, -73.24514590057679 9.40847309258136, -73.08168190042456 9.609445092768524, -73.00472690035288 9.768327092916508, -72.97750890032754 10.001945093134069, -72.93889990029157 10.11645409324072, -72.90354590025865 10.444445093546179, -72.49125489987468 11.12277309417793, -72.31764589971299 11.164509094216797, -72.20935489961214 11.25000009429641, -71.9684818993878 11.666245094684072, -71.7697268992027 11.700836094716294, -
 71.40333589886147 11.812773094820542, -71.32471789878825 11.853054094858052, -71.37861789883844 11.753336094765189, -71.44584589890106 11.723891094737766, -71.76583589919908 11.662364094680456, -71.95417289937447 11.594718094617463, -71.96639989938586 11.506391094535203, -71.93278189935455 11.363054094401704, -71.7408458991758 11.034164094095402, -71.763617899197 11.013745094076384, -71.69111789912948 10.834164093909138, -71.6618088991022 10.77750009385636, -71.57792689902406 10.71610909379919, -71.58168189902757 10.674718093760646, -71.64889989909017 10.442500093544368, -71.75569989918964 10.369718093476592, -71.79444589922572 10.321809093431966, -71.8411268992692 10.218891093336126, -72.1253548995339 9.818191092962934, -72.07792689948973 9.735554092885977, -71.97917289939775 9.623336092781472, -71.91084589933412 9.493609092660648, -71.73319989916868 9.375691092550824, -71.72361789915975 9.345000092522241, -71.75945489919313 9.118327092311148, -71.7148638991516 9.077918092273507, -
 71.62389089906688 9.043054092241036, -71.55278189900065 9.040836092238976, -71.3136088987779 9.1105540923039, -71.24139089871065 9.155418092345684, -71.16541789863989 9.273745092455883, -71.07486389855555 9.312500092491973, -71.05597289853796 9.338745092516419, -71.08917289856888 9.536600092700695, -71.05334589853551 9.703609092856226, -71.07339989855419 9.851109092993596, -71.26945489873678 10.150136093272081, -71.38556389884492 10.282773093395619, -71.43667289889251 10.36944509347633, -71.45861789891295 10.460000093560666, -71.54570889899406 10.568327093661566, -71.52639989897608 10.726945093809277, -71.44667289890182 10.795554093873179, -71.45167289890648 10.916391093985723, -71.49285489894484 10.961036094027293, -71.4119548988695 10.984164094048836, -71.28110889874763 10.989164094053493, -70.82556389832338 11.211391094260463, -70.50847289802806 11.248191094294725, -70.4466728979705 11.290000094333664, -70.23918189777727 11.353054094392391, -70.14249989768723 11.418336094453196, 
 -70.03486389758697 11.441318094474596, -70.02389989757677 11.491391094521234, -70.04779089759901 11.517773094545802, -69.987645897543 11.514718094542957, -69.87402689743719 11.426527094460823, -69.80139089736954 11.427218094461466, -69.7423998973146 11.499200094528504, -69.81653589738364 11.690973094707104, -70.18000889772216 11.603609094625739, -70.23598189777428 11.62889109464929, -70.24389989778166 11.775273094785618, -70.29403589782835 11.861664094866072, -70.28666389782148 11.92027309492066, -70.18749989772914 12.107773095095283, -70.01430889756783 12.197500095178839, -69.93471789749371 12.169718095152973, -69.85945489742362 12.071945095061906, -69.82375489739037 11.98805409498378, -69.81688189738396 11.850973094856116, -69.75778189732893 11.661391094679558, -69.7061178972808 11.548891094574785, -69.6318088972116 11.467636094499099, -69.5795178971629 11.46410009449582, -69.50695489709533 11.506945094535709, -69.36021789695867 11.493327094523039, -69.27487289687919 11.5339540945
 60869, -68.95389989658025 11.45166409448423, -68.8436818964776 11.447082094479967, -68.66042689630693 11.349864094389417, -68.60196389625249 11.290554094334183, -68.41833589608147 11.180000094231218, -68.24250889591772 10.874373093946588, -68.25195489592652 10.856664093930092, -68.32529089599481 10.843609093917934, -68.3280638959974 10.767500093847048, -68.25361789592806 10.588891093680715, -68.16514589584567 10.498891093596896, -67.86798189556892 10.464873093565217, -67.79610889550197 10.491945093590417, -67.54333589526657 10.53277309362845, -67.28029089502158 10.546664093641382, -67.00222689476261 10.610282093700633, -66.47056389426747 10.629164093718217, -66.34299089414866 10.604718093695453, -66.3102908941182 10.626018093715288)), ((-64.05555489201832 10.857218093930612, -64.1363998920936 10.945000094012357, -64.17403589212866 10.960418094026721, -64.22792689217884 10.93124509399955, -64.40569989234442 10.970136094035766, -64.37834589231893 11.056945094116614, -64.20180889215452
  11.08778209414534, -64.17584589213034 11.031664094093074, -64.10583589206514 10.995282094059192, -64.04278189200642 10.987636094052064, -63.98749989195494 11.076109094134466, -63.88493589185941 11.175618094227147, -63.84486389182209 11.127082094181944, -63.80472689178471 11.021391094083512, -63.81499989179429 10.978054094043145, -63.89000889186414 10.904445093974587, -64.05555489201832 10.857218093930612)), ((-61.163608889324976 8.68832709191068, -61.16499988932628 8.71527309193577, -61.042499889212195 8.821109092034334, -60.86055488904273 8.853336092064353, -60.97610888915035 8.725827091945604, -61.163608889324976 8.68832709191068)), ((-61.246945889402596 8.474718091711736, -61.2943088894467 8.493327091729071, -61.405281889550054 8.485836091722092, -61.543545889678825 8.54868209178062, -61.42639088956972 8.58332709181289, -61.279445889432864 8.569718091800212, -61.26149988941614 8.552964091784602, -61.27829988943179 8.516736091750872, -61.2635998894181 8.499927091735216, -61.18507
 2889344966 8.496736091732245, -61.246945889402596 8.474718091711736)), ((-61.1291728892929 8.501664091736828, -61.26368188941818 8.510209091744784, -61.253063889408295 8.545273091777446, -61.21750888937518 8.572500091802794, -61.14912688931149 8.54089109177336, -61.083617889250476 8.609718091837465, -60.99806388917081 8.59694509182556, -60.99860888917131 8.557782091789093, -61.04055488921037 8.51416409174847, -61.1291728892929 8.501664091736828)), ((-60.91028188908905 8.894164092102372, -61.01972688919098 8.846945092058391, -61.046390889215814 8.843891092055557, -60.94167288911828 9.011664092211802, -60.880345889061175 9.026527092225649, -60.840563889024125 8.99832709219939, -60.85625488903874 8.947364092151915, -60.91028188908905 8.894164092102372)), ((-65.2811088931597 10.88028209395209, -65.39334589326423 10.906945093976915, -65.41611789328543 10.927218093995805, -65.36445489323732 10.969164094034866, -65.30445489318144 10.976109094041334, -65.21215489309549 10.954164094020896, -
 65.19959089308378 10.898336093968908, -65.2811088931597 10.88028209395209)), ((-61.048054889217354 8.639718091865404, -61.146945889309464 8.654445091879111, -61.17722688933766 8.67666409189981, -60.93778188911466 8.721245091941327, -60.99680888916963 8.652636091877426, -61.048054889217354 8.639718091865404)), ((-60.85831788904065 9.06495409226143, -60.94333588911984 9.022218092221635, -61.061390889229784 8.896945092104957, -61.096672889262635 8.89090009209933, -61.07062688923838 8.97131809217423, -60.947226889123456 9.054718092251903, -60.850417889033295 9.092982092287542, -60.85831788904065 9.06495409226143)), ((-60.92333588910121 8.618327091845487, -60.988617889162 8.635554091861522, -60.84159988902509 8.726945091946632, -60.80944588899514 8.716945091937319, -60.82333588900808 8.652364091877175, -60.92333588910121 8.618327091845487)))	
+ASM	American Samoa	MULTIPOLYGON (((-170.74389999137958 -14.375554929569248, -170.74941799138472 -14.373890929567693, -170.7664449914006 -14.363608929558112, -170.82322699145345 -14.323754929521002, -170.80917299144036 -14.308054929506383, -170.79765499142965 -14.299308929498238, -170.68167299132162 -14.258054929459817, -170.6640269913052 -14.255417929457352, -170.5679179912157 -14.254308929456329, -170.56187299121007 -14.269999929470941, -170.57861799122566 -14.279163929479466, -170.63726399128026 -14.289445929489048, -170.74389999137958 -14.375554929569248)))	
+COK	Cook Is.	MULTIPOLYGON (((-159.7469819811379 -21.25667293597779, -159.79363598118135 -21.252781935974156, -159.83251798121756 -21.248472935970142, -159.84000898122454 -21.23916393596147, -159.83471798121963 -21.199308935924364, -159.82723598121265 -21.189863935915568, -159.7883269811764 -21.187499935913365, -159.75613598114643 -21.192363935917896, -159.73292698112482 -21.22624593594945, -159.73916398113062 -21.2524999359739, -159.7469819811379 -21.25667293597779)), ((-157.92889097944467 -21.94083593661496, -157.94696397946151 -21.93958193661379, -157.96376397947716 -21.92041793659594, -157.96363597947703 -21.90819093658456, -157.95751797947133 -21.89528193657253, -157.94778197946226 -21.88805493656581, -157.921735979438 -21.88146393655967, -157.88384497940274 -21.925208936600413, -157.88766397940628 -21.936808936611214, -157.92889097944467 -21.94083593661496)), ((-158.11667297961955 -20.019172934825278, -158.12668197962887 -20.009445934816213, -158.13041797963237 -19.995281934803
 018, -158.1253999796277 -19.978472934787362, -158.11152697961478 -19.9713179347807, -158.09475497959914 -19.97417293478337, -158.0836359795888 -19.98444593479293, -158.0813909795867 -19.996390934804054, -158.08876397959358 -20.01069993481738, -158.0983549796025 -20.016108934822412, -158.11667297961955 -20.019172934825278)), ((-157.71304497924365 -19.857226934674443, -157.71887297924908 -19.85166393466926, -157.7408179792695 -19.817645934637582, -157.73919997926802 -19.807508934628146, -157.71229997924297 -19.77312693459612, -157.70376397923502 -19.836808934655437, -157.70864497923955 -19.853054934670567, -157.71304497924365 -19.857226934674443)), ((-163.16946398432535 -18.091945933030402, -163.17128198432704 -18.08417293302317, -163.15472698431162 -18.06145493300201, -163.15725498431397 -18.080554933019798, -163.1633269843196 -18.089445933028074, -163.16946398432535 -18.091945933030402)), ((-165.84167298681402 -10.890835926323845, -165.84834498682025 -10.8842359263177, -165.82765498
 680098 -10.881317926314978, -165.84167298681402 -10.890835926323845)))	
+PYF	French Polynesia	MULTIPOLYGON (((-149.1791999712959 -17.870835932824477, -149.25809097136937 -17.85278193280766, -149.27570897138577 -17.84631793280164, -149.28893597139808 -17.833054932789295, -149.29795497140648 -17.82083593277791, -149.3694729714731 -17.738890932701594, -149.3785999714816 -17.743608932705982, -149.42086397152096 -17.756108932717623, -149.47168197156827 -17.766390932727205, -149.48763597158313 -17.76597293272681, -149.59057297167902 -17.711390932675982, -149.63891797172403 -17.625972932596426, -149.64169097172663 -17.611945932583367, -149.64141797172636 -17.592781932565515, -149.63559097172094 -17.561672932536553, -149.63250897171807 -17.54999993252568, -149.62388197171003 -17.540699932517015, -149.58739097167606 -17.516181932494177, -149.4921089715873 -17.493754932473294, -149.44998197154808 -17.499445932478594, -149.41473597151526 -17.509172932487658, -149.37362697147697 -17.526945932504205, -149.3591639714635 -17.534454932511196, -149.34861797145368 -17.542
 090932518306, -149.3327639714389 -17.57028193254456, -149.32666397143322 -17.592226932564998, -149.32333597143014 -17.65249993262114, -149.3230909714299 -17.67110893263846, -149.3240359714308 -17.68778193265399, -149.32014497142717 -17.702154932667383, -149.3075089714154 -17.712635932677145, -149.29306397140192 -17.717499932681676, -149.2305639713437 -17.730281932693572, -149.2077909713225 -17.734172932697206, -149.18473597130105 -17.73139093269461, -149.1747089712917 -17.736117932699017, -149.1622269712801 -17.751390932713235, -149.1547269712731 -17.764445932725394, -149.14837297126718 -17.781954932741698, -149.1466819712656 -17.805699932763815, -149.15221797127077 -17.83361793278982, -149.15780897127598 -17.849726932804813, -149.16559097128322 -17.86360893281774, -149.1741639712912 -17.869163932822914, -149.1791999712959 -17.870835932824477)), ((-139.05474496186676 -9.859999925363809, -139.1089179619172 -9.846945925351648, -139.11913596192673 -9.843608925348533, -139.1289089619358
 2 -9.838890925344145, -139.1411089619472 -9.830835925336643, -139.15142696195682 -9.820554925327073, -139.16879096197297 -9.79375492530211, -139.1722549619762 -9.772917925282698, -139.16726396197154 -9.759999925270677, -139.04444496185715 -9.69541792521052, -139.00472696182018 -9.69694592521195, -138.97568196179313 -9.708608925222805, -138.9687639617867 -9.722363925235626, -138.9570729617758 -9.741390925253341, -138.8903179617136 -9.757226925268085, -138.85058196167662 -9.756108925267043, -138.83611796166315 -9.749863925261238, -138.82531796165307 -9.740272925252299, -138.81251796164116 -9.737917925250102, -138.8097549616386 -9.748054925259552, -138.81558196164403 -9.759445925270157, -138.8319729616593 -9.770835925280764, -138.84915496167528 -9.781117925290346, -138.89585496171878 -9.806254925313752, -138.91085496173275 -9.808335925315689, -138.9794269617966 -9.8149999253219, -138.99999996181577 -9.8149999253219, -139.01319996182806 -9.811390925318534, -139.02973596184347 -9.8072269
 25314652, -139.04862696186106 -9.808199925315563, -139.06133596187289 -9.818890925325519, -139.05474496186676 -9.859999925363809)), ((-140.17782696291272 -8.956390924522253, -140.18945496292355 -8.954163924520174, -140.2058359629388 -8.944726924511386, -140.22584496295744 -8.930417924498059, -140.2305639629618 -8.921390924489657, -140.25613596298564 -8.827363924402093, -140.25339096298308 -8.813054924388766, -140.2497549629797 -8.80278192437919, -140.2450269629753 -8.793890924370913, -140.2366639629675 -8.7838909243616, -140.2251449629568 -8.778190924356295, -140.06890896281126 -8.81167292438748, -140.04281796278696 -8.830281924404801, -140.01599096276198 -8.852845924425822, -140.01196396275824 -8.870972924442697, -140.01515496276122 -8.888472924458995, -140.0226639627682 -8.898054924467928, -140.07781796281955 -8.918054924486555, -140.0908359628317 -8.9220819244903, -140.10487296284475 -8.918754924487203, -140.1197179628586 -8.918890924487329, -140.1669639629026 -8.933054924500524,
  -140.17683596291178 -8.940135924507118, -140.17782696291272 -8.956390924522253)), ((-151.4444909734056 -16.904445931924457, -151.46543597342512 -16.902917931923028, -151.47512697343413 -16.89749993191799, -151.49169997344958 -16.849172931872985, -151.49777297345523 -16.784026931812306, -151.49136397344927 -16.759999931789935, -151.48751797344568 -16.7494459317801, -151.4725179734317 -16.73971793177104, -151.43098197339302 -16.745554931776482, -151.35111797331865 -16.84583593186987, -151.3509819733185 -16.860345931883387, -151.40032697336449 -16.888335931909452, -151.4444909734056 -16.904445931924457)), ((-149.85442697192474 -17.574445932548443, -149.86834497193772 -17.56833593254275, -149.87805497194674 -17.563617932538364, -149.9250449719905 -17.52528193250265, -149.93808197200266 -17.509726932488164, -149.941681972006 -17.497217932476516, -149.93725497200188 -17.484163932464355, -149.9227909719884 -17.477781932458413, -149.91113597197756 -17.476108932456853, -149.79462697186906 -
 17.466799932448183, -149.78529097186035 -17.47125493245234, -149.78321797185842 -17.487781932467726, -149.79849097187264 -17.52778193250498, -149.84596397191686 -17.572499932546634, -149.85442697192474 -17.574445932548443)))	
+UMI	Jarvis I.	MULTIPOLYGON (((-160.02114498139323 -0.3980549165516862, -160.02810898139973 -0.3980549165516862, -160.04349098141404 -0.3922179165462438, -160.0451639814156 -0.3801359165349964, -160.03391798140512 -0.3743089165295714, -160.0177909813901 -0.3747179165299457, -160.00946398138237 -0.3847179165392589, -160.0129269813856 -0.3958359165496148, -160.02114498139323 -0.3980549165516862)))	
+NIU	Niue	MULTIPOLYGON (((-169.89389099058795 -19.145554934011656, -169.9308819906224 -19.124445933991993, -169.9522359906423 -19.073335933944392, -169.93028199062184 -19.01360893388876, -169.89474499058875 -18.970281933848412, -169.88476399057944 -18.963335933841947, -169.86889999056467 -18.963617933842215, -169.8151359905146 -18.970272933848406, -169.78155499048333 -19.065281933936888, -169.79809999049874 -19.087226933957325, -169.82443599052326 -19.110835933979317, -169.85028199054733 -19.125835933993287, -169.8591639905556 -19.130554933997686, -169.88806399058254 -19.14444593401062, -169.89389099058795 -19.145554934011656)))	
+WSM	Samoa	MULTIPOLYGON (((-172.59649999310494 -13.509108928762302, -172.55193599306344 -13.497217928751226, -172.47528199299205 -13.479717928734928, -172.39111799291368 -13.464172928720458, -172.36083599288548 -13.460554928717087, -172.34835499287385 -13.461390928717861, -172.303417992832 -13.472154928727889, -172.28779999281744 -13.484163928739065, -172.22249999275664 -13.563054928812548, -172.20306399273855 -13.59194592883945, -172.1933639927295 -13.613335928859371, -172.1683549927062 -13.680972928922358, -172.16781799270572 -13.691390928932066, -172.21235499274718 -13.806526929039293, -172.225035992759 -13.808890929041496, -172.25849999279018 -13.804308929037234, -172.39333599291575 -13.791672929025466, -172.48599099300202 -13.806808929039562, -172.50890899302337 -13.806663929039416, -172.5279269930411 -13.802708929035745, -172.5744359930844 -13.765835929001398, -172.59002699309892 -13.739717928977072, -172.69109999319306 -13.626108928871261, -172.7516909932495 -13.57402692882276
 , -172.77282699326918 -13.54999992880039, -172.78002699327587 -13.532572928784148, -172.7725359932689 -13.517499928770121, -172.75558199325312 -13.51028192876339, -172.73890899323757 -13.508472928761705, -172.728635993228 -13.513335928766239, -172.65527299315968 -13.519172928771681, -172.59649999310494 -13.509108928762302)), ((-171.44198199202972 -14.057499929273035, -171.4652729920514 -14.052781929268633, -171.47998199206512 -14.050554929266568, -171.52055499210292 -14.047781929263977, -171.5461089921267 -14.050281929266305, -171.5889269921666 -14.052499929268379, -171.64837299222194 -14.050281929266305, -171.76809999233345 -14.035835929252855, -171.91113599246665 -14.012499929231126, -172.05059999259655 -13.912499929137994, -172.0584999926039 -13.903190929129323, -172.06475499260972 -13.87819092910604, -172.05975499260506 -13.866945929095564, -172.02932699257673 -13.840272929070721, -171.90670899246254 -13.806663929039416, -171.8839089924413 -13.805554929038394, -171.8222639923838
 8 -13.807499929040205, -171.7480819923148 -13.83167292906272, -171.6191359921947 -13.87860892910642, -171.44418199203176 -13.984445929204995, -171.43070899201922 -14.002363929221687, -171.42919999201783 -14.01625492923462, -171.44198199202972 -14.057499929273035)))	
+TKL	Tokelau	MULTIPOLYGON (((-171.84805499240792 -9.218890924766725, -171.85885499241797 -9.209654924758127, -171.86271799242155 -9.18089992473135, -171.85243599241198 -9.170626924721773, -171.84419099240432 -9.191108924740846, -171.84376399240392 -9.210835924759223, -171.84805499240792 -9.218890924766725)))	
+TON	Tonga	MULTIPOLYGON (((-175.1452909954787 -21.268063935988394, -175.18639999551698 -21.252781935974156, -175.31445499563625 -21.17999993590638, -175.3236359956448 -21.174445935901204, -175.33642699565672 -21.161672935889314, -175.35140899567065 -21.14333593587223, -175.35748199567632 -21.131663935861354, -175.35999999567866 -21.10083593583265, -175.35319999567233 -21.0887549358214, -175.34112699566109 -21.074863935808466, -175.3142999956361 -21.06417293579851, -175.3035549956261 -21.069726935803672, -175.27667299560105 -21.124999935855158, -175.2452359955718 -21.12978193585961, -175.1490359954822 -21.17680893590341, -175.13836399547225 -21.175835935902498, -175.1319549954663 -21.159163935886966, -175.12945499546396 -21.14472693587352, -175.12139999545644 -21.13375493586331, -175.10723599544326 -21.12806393585801, -175.09362699543058 -21.124999935855158, -175.06542699540432 -21.125835935855932, -175.0487639953888 -21.13694593586628, -175.0459909953862 -21.149163935877652, -175.051
 69999539154 -21.161672935889314, -175.12083599545593 -21.262090935982826, -175.13503599546914 -21.267781935988125, -175.1452909954787 -21.268063935988394)), ((-173.93920899435545 -18.56889093347459, -173.93362699435025 -18.573054933478474, -173.9166909943345 -18.599726933503305, -173.9068269943253 -18.62375493352569, -173.90890899432722 -18.635563933536687, -173.98697299439993 -18.684172933581962, -174.05444499446276 -18.669726933568498, -174.06308199447082 -18.659299933558785, -174.07030899447756 -18.63597293353706, -174.06527299447285 -18.62499993352685, -174.05334499446175 -18.621117933523237, -174.03737299444688 -18.621117933523237, -174.0252999944356 -18.616672933519098, -174.01724499442813 -18.603054933506414, -174.01306399442424 -18.589726933493992, -174.00321799441505 -18.578199933483262, -173.99249099440507 -18.572226933477694, -173.9675269943818 -18.568054933473817, -173.93920899435545 -18.56889093347459)))	
+WLF	Wallis & Futuna	MULTIPOLYGON (((-178.06081799819398 -14.323890929521127, -178.13739099826532 -14.317081929514785, -178.15389999828068 -14.308054929506383, -178.17110899829672 -14.287781929487494, -178.1793549983044 -14.275417929475978, -178.18832699831276 -14.256108929457994, -178.19027299831455 -14.239726929442739, -178.18169999830658 -14.232363929435891, -178.12733599825594 -14.24860892945101, -178.04301799817742 -14.31749992951518, -178.06081799819398 -14.323890929521127)), ((-176.1650359964284 -13.35305492861697, -176.16906399643216 -13.351945928615933, -176.17724499643978 -13.341390928606103, -176.18808199644988 -13.313608928580223, -176.1910999964527 -13.286945928555397, -176.18581799644775 -13.257781928528232, -176.18307299644522 -13.24360892851503, -176.1783449964408 -13.232226928504431, -176.1583549964222 -13.21486392848827, -176.14809099641263 -13.216108928489419, -176.12193599638826 -13.258608928529, -176.14202699640697 -13.34381792860836, -176.16143599642507 -13.3527
 72928616702, -176.1650359964284 -13.35305492861697)))	
+ARG	Argentina	MULTIPOLYGON (((-71.85916389928599 -41.0112819543757, -71.85028189927772 -40.91250895428371, -71.95014589937072 -40.72694595411089, -71.85930889928613 -40.64347295403315, -71.77987289921215 -40.40896395381475, -71.71569989915238 -40.42361795382839, -71.6639638991042 -40.33451795374541, -71.66861789910853 -40.29694595371042, -71.7027178991403 -40.27895495369366, -71.719590899156 -40.302363953715464, -71.81806389924772 -40.20459095362441, -71.79139989922288 -40.11500895354098, -71.66694589910698 -40.04749995347811, -71.63334589907568 -39.9505

<TRUNCATED>

[09/26] lucenenet git commit: Added missing Spatial classes, ported Spatial tests, and fixed several bugs

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Test-Files/Data/world-cities-points.txt
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Test-Files/Data/world-cities-points.txt b/src/Lucene.Net.Tests.Spatial/Test-Files/Data/world-cities-points.txt
new file mode 100644
index 0000000..59a7cd5
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Test-Files/Data/world-cities-points.txt
@@ -0,0 +1,2680 @@
+#id	name	shape	
+G292223	Dubai	POINT(55.280000 25.252220)	
+G292672	Sharjah	POINT(55.391110 25.362220)	
+G292913	Al \u2018Ayn	POINT(55.760560 24.191670)	
+G292932	`Ajm\u0101n	POINT(55.435040 25.411110)	
+G292968	Abu Dhabi	POINT(54.366670 24.466670)	
+G1133616	Maz\u0101r-e Shar\u012bf	POINT(67.110870 36.709040)	
+G1135689	Kunduz	POINT(68.857000 36.728960)	
+G1138336	Kandah\u0101r	POINT(65.710130 31.613320)	
+G1138958	Kabul	POINT(69.172330 34.528130)	
+G1139715	Jal\u0101l\u0101b\u0101d	POINT(70.451530 34.426470)	
+G1140026	Her\u0101t	POINT(62.199670 34.348170)	
+G3183875	Tirana	POINT(19.818890 41.327500)	
+G616052	Yerevan	POINT(44.513610 40.181110)	
+G2240449	Luanda	POINT(13.234440 -8.838330)	
+G3347939	Lobito	POINT(13.545560 -12.348060)	
+G3348313	Huambo	POINT(15.739170 -12.776110)	
+G3351663	Benguela	POINT(13.407220 -12.578330)	
+G3429577	Resistencia	POINT(-58.983330 -27.450000)	
+G3429652	Quilmes	POINT(-58.269440 -34.720280)	
+G3429886	Posadas	POINT(-55.883330 -27.383330)	
+G3430863	Mar del Plata	POINT(-57.550000 -38.000000)	
+G3432043	La Plata	POINT(-57.948890 -34.931390)	
+G3433899	Formosa	POINT(-58.183330 -26.183330)	
+G3435217	Corrientes	POINT(-58.833330 -27.466670)	
+G3435910	Buenos Aires	POINT(-58.377230 -34.613150)	
+G3835869	Santiago del Estero	POINT(-64.266670 -27.783330)	
+G3836277	Santa Fe de la Vera Cruz	POINT(-60.700000 -31.633330)	
+G3836564	San Salvador de Jujuy	POINT(-65.300000 -24.183330)	
+G3836873	San Miguel de Tucum�n	POINT(-65.216670 -26.816670)	
+G3837056	San Luis	POINT(-66.350000 -33.300000)	
+G3837213	San Juan	POINT(-68.536390 -31.537500)	
+G3837702	San Fernando del Valle de Catamarca	POINT(-65.783330 -28.466670)	
+G3838233	Salta	POINT(-65.416670 -24.783330)	
+G3838583	Rosario	POINT(-60.666390 -32.951110)	
+G3838874	R�o Cuarto	POINT(-64.350000 -33.133330)	
+G3841956	Paran�	POINT(-60.533330 -31.733330)	
+G3843123	Neuqu�n	POINT(-68.066670 -38.950000)	
+G3844421	Mendoza	POINT(-68.816670 -32.883330)	
+G3848950	La Rioja	POINT(-66.850000 -29.433330)	
+G3860259	C�rdoba	POINT(-64.183330 -31.400000)	
+G3865086	Bah�a Blanca	POINT(-62.283330 -38.716670)	
+G2761369	Vienna	POINT(16.372080 48.208490)	
+G2766824	Salzburg	POINT(13.043990 47.799410)	
+G2772400	Linz	POINT(14.286110 48.306390)	
+G2778067	Graz	POINT(15.450000 47.066670)	
+G7279909	Linz-Wels-Steyr	POINT(14.183690 48.154580)	
+G2063523	Perth	POINT(115.833330 -31.933330)	
+G2078025	Adelaide	POINT(138.600000 -34.933330)	
+G2147714	Sydney	POINT(151.207320 -33.867850)	
+G2155472	Newcastle	POINT(151.776470 -32.927150)	
+G2158177	Melbourne	POINT(144.963320 -37.814000)	
+G2165087	Gold Coast	POINT(153.433330 -28.000000)	
+G2165796	Geelong West	POINT(144.350000 -38.133330)	
+G2165798	Geelong	POINT(144.360690 -38.147110)	
+G2171507	Wollongong	POINT(150.883330 -34.433330)	
+G2172517	Canberra	POINT(149.128070 -35.283460)	
+G2172797	Cairns	POINT(145.766670 -16.916670)	
+G2174003	Brisbane	POINT(153.028090 -27.467940)	
+G7280463	North Shore	POINT(152.901850 -31.402370)	
+G7281838	Logan City	POINT(153.109440 -27.639170)	
+G584923	Sumqay\u0131t	POINT(49.668610 40.589720)	
+G586523	Kirowabad	POINT(46.360560 40.682780)	
+G587084	Baku	POINT(49.892010 40.377670)	
+G3186573	Zenica	POINT(17.906390 44.201390)	
+G3191281	Sarajevo	POINT(18.356440 43.848640)	
+G3204541	Banja Luka	POINT(17.185560 44.775830)	
+G1185098	Tungi	POINT(90.405830 23.890000)	
+G1185099	Sylhet	POINT(91.871670 24.896670)	
+G1185106	Jam\u0101lpur	POINT(89.933330 24.916670)	
+G1185117	Narsingdi	POINT(90.718060 23.920830)	
+G1185128	R\u0101jsh\u0101hi	POINT(88.600000 24.366670)	
+G1185155	N\u0101r\u0101yanganj	POINT(90.501110 23.623330)	
+G1185162	Mymensingh	POINT(90.400000 24.750000)	
+G1185186	Comilla	POINT(91.204440 23.457780)	
+G1185188	Rangpur	POINT(89.250000 25.750000)	
+G1185241	Dhaka	POINT(90.407440 23.710400)	
+G1192366	P\u0101r Naogaon	POINT(88.950840 24.802870)	
+G1203891	Din\u0101jpur	POINT(88.638640 25.627150)	
+G1205733	Chittagong	POINT(91.836390 22.333060)	
+G1336134	Cox\u2019s B\u0101z\u0101r	POINT(91.979770 21.453240)	
+G1336135	Khulna	POINT(89.567230 22.813480)	
+G1336137	Baris\u0101l	POINT(90.371110 22.701940)	
+G1336140	Jessore	POINT(89.213150 23.168690)	
+G1336144	T\u0101ng\u0101il	POINT(89.916670 24.250000)	
+G1337233	Bogra	POINT(89.366670 24.850000)	
+G6545349	Saidpur	POINT(88.891690 25.777690)	
+G2792413	Li�ge	POINT(5.567490 50.633730)	
+G2797656	Gent	POINT(3.716670 51.050000)	
+G2800481	Charleroi	POINT(4.452480 50.409470)	
+G2800866	Brussels	POINT(4.348780 50.850450)	
+G2803138	Antwerpen	POINT(4.403460 51.219890)	
+G2357048	Ouagadougou	POINT(-1.538340 12.364230)	
+G2362344	Bobo-Dioulasso	POINT(-4.297900 11.177150)	
+G726050	Varna	POINT(27.916670 43.216670)	
+G727011	Sofia	POINT(23.324150 42.697510)	
+G727523	Ruse	POINT(25.970830 43.856390)	
+G728193	Plovdiv	POINT(24.750000 42.150000)	
+G732770	Burgas	POINT(27.467810 42.506060)	
+G425378	Bujumbura	POINT(29.364400 -3.382200)	
+G2392087	Porto-Novo	POINT(2.603590 6.496460)	
+G2392204	Parakou	POINT(2.616670 9.350000)	
+G2394560	Djougou	POINT(1.666940 9.705000)	
+G2394819	Cotonou	POINT(2.433330 6.350000)	
+G2395914	Abomey-Calavi	POINT(2.350000 6.450000)	
+G3903320	Tarija	POINT(-64.729560 -21.535490)	
+G3903987	Sucre	POINT(-65.262740 -19.033320)	
+G3904906	Santa Cruz de la Sierra	POINT(-63.166670 -17.800000)	
+G3909234	Oruro	POINT(-67.150000 -17.983330)	
+G3911925	La Paz	POINT(-68.150000 -16.500000)	
+G3919968	Cochabamba	POINT(-66.156800 -17.389500)	
+G3386496	Teresina	POINT(-42.801940 -5.089170)	
+G3387296	Sobral	POINT(-40.349720 -3.686110)	
+G3388368	S�o Lu�s	POINT(-44.302780 -2.529720)	
+G3389353	Santar�m	POINT(-54.708330 -2.443060)	
+G3390760	Recife	POINT(-34.881110 -8.053890)	
+G3392242	Petrolina	POINT(-40.500830 -9.398610)	
+G3392740	Paulista	POINT(-34.873060 -7.940830)	
+G3392998	Parnamirim	POINT(-35.262780 -5.915560)	
+G3393536	Olinda	POINT(-34.855280 -8.008890)	
+G3394023	Natal	POINT(-35.209440 -5.795000)	
+G3394682	Mossor�	POINT(-37.344170 -5.187500)	
+G3395473	Maracana�	POINT(-38.625560 -3.876670)	
+G3395981	Macei�	POINT(-35.735280 -9.665830)	
+G3396016	Macap�	POINT(-51.066390 0.038890)	
+G3397147	Juazeiro do Norte	POINT(-39.315280 -7.213060)	
+G3397277	Jo�o Pessoa	POINT(-34.863060 -7.115000)	
+G3397838	Jaboat�o	POINT(-35.001390 -8.180280)	
+G3398269	Imperatriz	POINT(-47.491670 -5.526390)	
+G3399415	Fortaleza	POINT(-38.543060 -3.717220)	
+G3402429	Caucaia	POINT(-38.653060 -3.736110)	
+G3402655	Caruaru	POINT(-35.976110 -8.283330)	
+G3403642	Campina Grande	POINT(-35.881110 -7.230560)	
+G3405870	Bel�m	POINT(-48.504440 -1.455830)	
+G3407327	Arapiraca	POINT(-36.661110 -9.752500)	
+G3407669	Ananindeua	POINT(-48.372220 -1.365560)	
+G3444876	Volta Redonda	POINT(-44.104170 -22.523060)	
+G3444914	Vit�ria da Conquista	POINT(-40.839440 -14.866110)	
+G3444924	Vit�ria	POINT(-40.337780 -20.319440)	
+G3445026	Vila Velha	POINT(-40.292500 -20.329720)	
+G3445156	Viam�o	POINT(-51.023330 -30.081110)	
+G3445451	V�rzea Grande	POINT(-56.132500 -15.646670)	
+G3445831	Uberl�ndia	POINT(-48.277220 -18.918610)	
+G3445839	Uberaba	POINT(-47.931940 -19.748330)	
+G3446682	Taubat�	POINT(-45.555280 -23.026390)	
+G3447186	Tabo�o da Serra	POINT(-46.791670 -23.626110)	
+G3447212	Suzano	POINT(-46.310830 -23.542500)	
+G3447259	Sumar�	POINT(-47.266940 -22.821940)	
+G3447399	Sorocaba	POINT(-47.458060 -23.501670)	
+G3447624	Sete Lagoas	POINT(-44.246670 -19.465830)	
+G3447779	Serra	POINT(-40.307780 -20.128610)	
+G3448136	S�o Vicente	POINT(-46.391940 -23.963060)	
+G3448439	S�o Paulo	POINT(-46.636110 -23.547500)	
+G3448622	S�o Leopoldo	POINT(-51.147220 -29.760280)	
+G3448636	S�o Jos� dos Campos	POINT(-45.886940 -23.179440)	
+G3448639	S�o Jos� do Rio Preto	POINT(-49.379440 -20.819720)	
+G3448742	S�o Jos�	POINT(-49.166670 -28.233330)	
+G3448877	S�o Jo�o de Meriti	POINT(-43.372220 -22.803890)	
+G3449319	S�o Carlos	POINT(-47.890830 -22.017500)	
+G3449344	S�o Bernardo do Campo	POINT(-46.565000 -23.693890)	
+G3449433	Santos	POINT(-46.333610 -23.960830)	
+G3449701	Santo Andr�	POINT(-46.538330 -23.663890)	
+G3450083	Santa Maria	POINT(-53.806940 -29.684170)	
+G3450144	Santa Luzia	POINT(-43.851390 -19.769720)	
+G3450404	Santa B�rbara d'Oeste	POINT(-47.413610 -22.753610)	
+G3450554	Salvador	POINT(-38.510830 -12.971110)	
+G3450909	Rondon�polis	POINT(-54.635560 -16.470830)	
+G3451138	Rio Grande	POINT(-52.098610 -32.035000)	
+G3451190	Rio de Janeiro	POINT(-43.207500 -22.902780)	
+G3451234	Rio Claro	POINT(-47.561390 -22.411390)	
+G3451328	Ribeir�o Preto	POINT(-47.810280 -21.177500)	
+G3451353	Ribeir�o das Neves	POINT(-44.086670 -19.766940)	
+G3452324	Presidente Prudente	POINT(-51.388890 -22.125560)	
+G3452465	Praia Grande	POINT(-46.402780 -24.005830)	
+G3452925	Porto Alegre	POINT(-51.230000 -30.033060)	
+G3453186	Ponta Grossa	POINT(-50.161940 -25.095000)	
+G3453643	Piracicaba	POINT(-47.649170 -22.725280)	
+G3454031	Petr�polis	POINT(-43.178610 -22.505000)	
+G3454244	Pelotas	POINT(-52.342500 -31.771940)	
+G3454857	Passo Fundo	POINT(-52.406670 -28.262780)	
+G3455775	Osasco	POINT(-46.791670 -23.532500)	
+G3456068	Novo Hamburgo	POINT(-51.130560 -29.678330)	
+G3456160	Nova Igua�u	POINT(-43.451110 -22.759170)	
+G3456166	Nova Friburgo	POINT(-42.531110 -22.281940)	
+G3456223	Nossa Senhora do Socorro	POINT(-37.126110 -10.855000)	
+G3456283	Niter�i	POINT(-43.103610 -22.883330)	
+G3456814	Montes Claros	POINT(-43.861670 -16.735000)	
+G3457001	Mogi das Cruzes	POINT(-46.188330 -23.522780)	
+G3457381	Mau�	POINT(-46.461390 -23.667780)	
+G3457671	Maring�	POINT(-51.938610 -23.425280)	
+G3457692	Mar�lia	POINT(-49.945830 -22.213890)	
+G3458449	Londrina	POINT(-51.162780 -23.310280)	
+G3458575	Limeira	POINT(-47.401670 -22.564720)	
+G3458930	Lages	POINT(-50.326110 -27.816110)	
+G3459462	Jundia�	POINT(-46.884170 -23.186390)	
+G3459505	Juiz de Fora	POINT(-43.350280 -21.764170)	
+G3459712	Joinville	POINT(-48.845560 -26.304440)	
+G3460370	Jacare�	POINT(-45.965830 -23.305280)	
+G3460644	Itaquaquecetuba	POINT(-46.348330 -23.486110)	
+G3460718	Itapevi	POINT(-46.934170 -23.548890)	
+G3460748	Itapecerica da Serra	POINT(-46.849170 -23.716940)	
+G3460845	Itaja�	POINT(-48.661940 -26.907780)	
+G3460949	Itabuna	POINT(-39.280280 -14.785560)	
+G3460950	Itabora�	POINT(-42.859440 -22.744440)	
+G3461144	Ipatinga	POINT(-42.536670 -19.468330)	
+G3461311	Indaiatuba	POINT(-47.218060 -23.090280)	
+G3461408	Ilh�us	POINT(-39.049440 -14.788890)	
+G3461655	Hortol�ndia	POINT(-47.220000 -22.858330)	
+G3461786	Guarulhos	POINT(-46.533330 -23.462780)	
+G3461789	Guaruj�	POINT(-46.256390 -23.993060)	
+G3461879	Guarapuava	POINT(-51.458060 -25.395280)	
+G3462089	Gravata�	POINT(-50.991940 -29.944440)	
+G3462315	Governador Valadares	POINT(-41.949440 -18.851110)	
+G3462377	Goi�nia	POINT(-49.253890 -16.678610)	
+G3462980	Francisco Morato	POINT(-46.745280 -23.281670)	
+G3463011	Franca	POINT(-47.400830 -20.538610)	
+G3463030	Foz do Igua�u	POINT(-54.588060 -25.547780)	
+G3463237	Florian�polis	POINT(-48.549170 -27.596670)	
+G3463422	Ferraz de Vasconcelos	POINT(-46.368610 -23.540830)	
+G3463478	Feira de Santana	POINT(-38.966670 -12.266670)	
+G3464305	Embu	POINT(-46.852220 -23.648890)	
+G3464374	Duque de Caxias	POINT(-43.311670 -22.785560)	
+G3464460	Dourados	POINT(-54.805560 -22.221110)	
+G3464688	Divin�polis	POINT(-44.883890 -20.138890)	
+G3464739	Diadema	POINT(-46.622780 -23.686110)	
+G3464975	Curitiba	POINT(-49.273060 -25.427780)	
+G3465038	Cuiab�	POINT(-56.096670 -15.596110)	
+G3465196	Crici�ma	POINT(-49.369720 -28.677500)	
+G3465284	Cotia	POINT(-46.919170 -23.603890)	
+G3465624	Contagem	POINT(-44.053610 -19.931670)	
+G3465927	Colombo	POINT(-49.224170 -25.291670)	
+G3466296	Chapec�	POINT(-52.618330 -27.096390)	
+G3466537	Caxias do Sul	POINT(-51.179440 -29.168060)	
+G3466779	Cascavel	POINT(-53.455280 -24.955830)	
+G3466998	Carapicu�ba	POINT(-46.835560 -23.522500)	
+G3467467	Canoas	POINT(-51.183610 -29.917780)	
+G3467693	Campos	POINT(-41.300000 -21.750000)	
+G3467747	Campo Grande	POINT(-54.646390 -20.442780)	
+G3467865	Campinas	POINT(-47.060830 -22.905560)	
+G3468031	Cama�ari	POINT(-38.324170 -12.697500)	
+G3468376	Cachoeiro de Itapemirim	POINT(-41.112780 -20.848890)	
+G3469058	Bras�lia	POINT(-47.929720 -15.779720)	
+G3469968	Blumenau	POINT(-49.066110 -26.919440)	
+G3470044	Betim	POINT(-44.198330 -19.967780)	
+G3470127	Belo Horizonte	POINT(-43.937780 -19.920830)	
+G3470142	Belford Roxo	POINT(-43.399440 -22.764170)	
+G3470279	Bauru	POINT(-49.060560 -22.314720)	
+G3470353	Barueri	POINT(-46.876110 -23.510560)	
+G3470583	Barreiras	POINT(-44.990000 -12.152780)	
+G3470636	Barra Mansa	POINT(-44.171390 -22.544170)	
+G3471766	Araraquara	POINT(-48.175560 -21.794440)	
+G3471859	Ara�atuba	POINT(-50.432780 -21.208890)	
+G3471872	Aracaju	POINT(-37.071670 -10.911110)	
+G3472177	Angra dos Reis	POINT(-44.318060 -23.006670)	
+G3472287	An�polis	POINT(-48.952780 -16.326670)	
+G3472343	Americana	POINT(-47.331390 -22.739170)	
+G3474574	Palmas	POINT(-48.360280 -10.212780)	
+G3662574	Rio Branco	POINT(-67.810000 -9.974720)	
+G3662762	Porto Velho	POINT(-63.903890 -8.761940)	
+G3663517	Manaus	POINT(-60.025000 -3.101940)	
+G3664980	Boa Vista	POINT(-60.673330 2.819720)	
+G6316406	Aparecida de Goi�nia	POINT(-49.243890 -16.823330)	
+G6317344	Jaboat�o dos Guararapes	POINT(-35.014720 -8.112780)	
+G3571824	Nassau	POINT(-77.343060 25.058230)	
+G933773	Gaborone	POINT(25.908590 -24.654510)	
+G620127	Vitsyebsk	POINT(30.194440 55.192500)	
+G625144	Minsk	POINT(27.566670 53.900000)	
+G625665	Mahilyow	POINT(30.336390 53.913890)	
+G627904	Hrodna	POINT(23.814720 53.681390)	
+G627907	Homyel\u2019	POINT(30.983330 52.441670)	
+G629634	Brest	POINT(23.700000 52.100000)	
+G630429	Baranavichy	POINT(26.033330 53.133330)	
+G630468	Babruysk	POINT(29.233330 53.150000)	
+G5881791	Abbotsford	POINT(-122.252570 49.057980)	
+G5894171	Barrie	POINT(-79.666340 44.400110)	
+G5907364	Brampton	POINT(-79.766330 43.683410)	
+G5911592	Burlington	POINT(-79.837130 43.386210)	
+G5911606	Burnaby	POINT(-122.952630 49.266360)	
+G5913490	Calgary	POINT(-114.085290 51.050110)	
+G5946768	Edmonton	POINT(-113.468710 53.550140)	
+G5959974	Gatineau	POINT(-75.701640 45.477230)	
+G5964700	Greater Sudbury	POINT(-80.990010 46.490000)	
+G5969785	Hamilton	POINT(-79.949640 43.233410)	
+G5992996	Kitchener	POINT(-80.482990 43.450100)	
+G6050610	Laval	POINT(-73.692000 45.569950)	
+G6058560	London	POINT(-81.233040 42.983390)	
+G6059891	Longueuil	POINT(-73.518060 45.531210)	
+G6075357	Mississauga	POINT(-79.658300 43.578900)	
+G6077243	Montr�al	POINT(-73.587810 45.508840)	
+G6091104	North York	POINT(-79.416300 43.766810)	
+G6094578	Oshawa	POINT(-78.849570 43.900120)	
+G6094817	Ottawa	POINT(-75.698120 45.411170)	
+G6119109	Regina	POINT(-104.617800 50.450080)	
+G6122085	Richmond	POINT(-123.136830 49.170030)	
+G6141256	Saskatoon	POINT(-106.634520 52.116790)	
+G6159905	Surrey	POINT(-122.825090 49.106350)	
+G6167865	Toronto	POINT(-79.416300 43.700110)	
+G6173331	Vancouver	POINT(-123.119340 49.249660)	
+G6173577	Vaughan	POINT(-79.532910 43.833410)	
+G6174041	Victoria	POINT(-123.369300 48.432940)	
+G6182962	Windsor	POINT(-83.016540 42.300080)	
+G6183235	Winnipeg	POINT(-97.147040 49.884400)	
+G6324729	Halifax	POINT(-63.572390 44.645330)	
+G6325494	Qu�bec	POINT(-71.214540 46.812280)	
+G7602078	Ladner	POINT(-123.082410 49.089380)	
+G204405	Uvira	POINT(29.145830 -3.406670)	
+G204953	Tshikapa	POINT(20.800000 -6.416670)	
+G207570	Mwene-Ditu	POINT(23.450000 -7.000000)	
+G209228	Mbuji-Mayi	POINT(23.600000 -6.150000)	
+G212730	Kisangani	POINT(25.200000 0.516670)	
+G214481	Kananga	POINT(22.417780 -5.895830)	
+G216449	Gandajika	POINT(23.950000 -6.750000)	
+G217562	Butembo	POINT(29.283330 0.150000)	
+G217831	Bukavu	POINT(28.860830 -2.508330)	
+G922704	Lubumbashi	POINT(27.466670 -11.666670)	
+G922741	Likasi	POINT(26.733330 -10.981390)	
+G922773	Kolwezi	POINT(25.472500 -10.716670)	
+G2312895	Mbandaka	POINT(18.266670 0.066670)	
+G2313002	Matadi	POINT(13.450000 -5.816670)	
+G2314302	Kinshasa	POINT(15.321460 -4.324590)	
+G2314705	Kikwit	POINT(18.818060 -5.038610)	
+G2593460	Masina	POINT(15.391390 -4.383610)	
+G2389853	Bangui	POINT(18.554960 4.361220)	
+G2255414	Pointe-Noire	POINT(11.846110 -4.794720)	
+G2260535	Brazzaville	POINT(15.283270 -4.266900)	
+G2657896	Z�rich	POINT(8.550000 47.366670)	
+G2660646	Gen�ve	POINT(6.145690 46.202220)	
+G2661604	Basel	POINT(7.600000 47.566670)	
+G2279755	Yamoussoukro	POINT(-5.283330 6.816670)	
+G2282006	San-P�dro	POINT(-6.616670 4.733330)	
+G2286304	Korhogo	POINT(-5.633330 9.450000)	
+G2290486	Daloa	POINT(-6.451940 6.874720)	
+G2290956	Bouak�	POINT(-5.033060 7.683330)	
+G2293521	Abobo	POINT(-4.020560 5.418890)	
+G2293538	Abidjan	POINT(-4.028060 5.341110)	
+G3868121	Vi�a del Mar	POINT(-71.551830 -33.024570)	
+G3868626	Valpara�so	POINT(-71.627250 -33.039320)	
+G3870011	Temuco	POINT(-72.600000 -38.733330)	
+G3870282	Talcahuano	POINT(-73.116670 -36.716670)	
+G3870294	Talca	POINT(-71.666670 -35.433330)	
+G3871336	Santiago	POINT(-70.566560 -33.426280)	
+G3872348	San Bernardo	POINT(-70.716670 -33.600000)	
+G3873775	Rancagua	POINT(-70.744440 -34.170830)	
+G3874960	Puerto Montt	POINT(-72.936940 -41.471670)	
+G3875024	Puente Alto	POINT(-70.583330 -33.616670)	
+G3884373	La Serena	POINT(-71.254170 -29.907780)	
+G3887127	Iquique	POINT(-70.143060 -20.220830)	
+G3893629	Coquimbo	POINT(-71.343610 -29.953330)	
+G3893894	Concepci�n	POINT(-73.049770 -36.826990)	
+G3895088	Chill�n	POINT(-72.103440 -36.606640)	
+G3899361	Arica	POINT(-70.304170 -18.475000)	
+G3899539	Antofagasta	POINT(-70.400000 -23.650000)	
+G7281017	La Pintana	POINT(-70.634190 -33.583310)	
+G2220957	Yaound�	POINT(11.516670 3.866670)	
+G2224827	Ngaound�r�	POINT(13.583330 7.316670)	
+G2226275	Mokolo	POINT(13.801880 10.739780)	
+G2228373	Maroua	POINT(14.315920 10.590950)	
+G2229152	Loum	POINT(9.735100 4.718200)	
+G2229798	Kouss�ri	POINT(15.030630 12.076890)	
+G2231320	Garoua	POINT(13.400000 9.300000)	
+G2232239	Ed�a	POINT(10.133330 3.800000)	
+G2232593	Douala	POINT(9.708400 4.046900)	
+G2234359	Bertoua	POINT(13.683330 4.583330)	
+G2234974	Bamenda	POINT(10.158240 5.952660)	
+G2235189	Bafoussam	POINT(10.417860 5.473660)	
+G1280849	Kashi	POINT(75.979720 39.454720)	
+G1529102	Urunchi	POINT(87.583330 43.800000)	
+G1529114	Turpan	POINT(89.166670 42.933330)	
+G1529195	Shihezi	POINT(86.033330 44.300000)	
+G1529376	Korla	POINT(86.146940 41.759720)	
+G1529569	Changji	POINT(87.316670 44.016670)	
+G1529641	Aral	POINT(81.263610 40.515560)	
+G1529660	Aksu	POINT(80.264440 41.123060)	
+G1783621	Zunyi	POINT(106.907220 27.686670)	
+G1783633	Zoucheng	POINT(116.965560 35.400560)	
+G1783745	Zigong	POINT(104.776890 29.341620)	
+G1783763	Zhuzhou	POINT(113.150000 27.833330)	
+G1783873	Zhumadian	POINT(114.029440 32.979440)	
+G1783934	Shangqiu	POINT(115.650000 34.450000)	
+G1784130	Zhoukou	POINT(114.633330 33.633330)	
+G1784554	Zhicheng	POINT(111.504720 30.295560)	
+G1784580	Yizheng	POINT(119.178890 32.269170)	
+G1784642	Zhenjiang	POINT(119.434170 32.209170)	
+G1784658	Zhengzhou	POINT(113.648610 34.757780)	
+G1784853	Zhaoqing	POINT(112.459720 23.051160)	
+G1784990	Zhanjiang	POINT(110.342710 21.281450)	
+G1785018	Zhangzhou	POINT(117.655560 24.513330)	
+G1785286	Zibo	POINT(118.063330 36.790560)	
+G1785294	Anyang	POINT(114.328890 36.099440)	
+G1785453	Zaozhuang	POINT(117.554170 34.864720)	
+G1785462	Zaoyang	POINT(112.754170 32.127220)	
+G1785725	Yunfu	POINT(112.037300 22.930560)	
+G1785738	Yuncheng	POINT(110.992780 35.023060)	
+G1785974	Yuci	POINT(112.731940 37.680280)	
+G1786067	Yuanlin	POINT(112.885950 30.415130)	
+G1786640	Yingcheng	POINT(113.550000 30.950000)	
+G1786657	Yinchuan	POINT(106.273060 38.468060)	
+G1786746	Yichun	POINT(114.400000 27.833330)	
+G1786764	Yichang	POINT(111.284720 30.714440)	
+G1786770	Yibin	POINT(104.623830 28.766670)	
+G1787093	Yantai	POINT(121.400000 37.533330)	
+G1787227	Yangzhou	POINT(119.435830 32.397220)	
+G1787323	Yangshuo	POINT(110.489670 24.780810)	
+G1787351	Yangquan	POINT(113.563330 37.857500)	
+G1787746	Yancheng	POINT(120.125280 33.385560)	
+G1787824	Tongshan	POINT(117.157070 34.180450)	
+G1787858	Xuri	POINT(117.966670 28.466670)	
+G1788046	Xuchang	POINT(113.816670 34.016670)	
+G1788450	Xinzhou	POINT(112.733330 38.409170)	
+G1788534	Xinyang	POINT(114.065560 32.122780)	
+G1788572	Xinxiang	POINT(113.867220 35.308890)	
+G1788618	Xintai	POINT(117.751940 35.900560)	
+G1788694	Xinpu	POINT(119.159440 34.599720)	
+G1788852	Xining	POINT(101.766670 36.616670)	
+G1788927	Xingtai	POINT(114.494170 37.063060)	
+G1789137	Xindi	POINT(113.466670 29.816670)	
+G1789273	Sanshui	POINT(112.891610 23.154860)	
+G1790254	Xiaogan	POINT(113.900000 30.916670)	
+G1790353	Xianyang	POINT(108.702610 34.337780)	
+G1790371	Xiantao	POINT(113.400000 30.383330)	
+G1790396	Xianning	POINT(114.216670 29.883330)	
+G1790437	Zhuhai	POINT(113.567780 22.276940)	
+G1790492	Xiangtan	POINT(112.900000 27.850000)	
+G1790587	Xiangfan	POINT(112.145000 32.041670)	
+G1790630	Xi\u2019an	POINT(108.928610 34.258330)	
+G1790645	Xiamen	POINT(118.081870 24.479790)	
+G1790840	Wuzhou	POINT(111.316670 23.483330)	
+G1790894	Wuxue	POINT(115.552500 29.850580)	
+G1790923	Wuxi	POINT(120.288570 31.568870)	
+G1791121	Changde	POINT(111.678060 29.032220)	
+G1791236	Wuhu	POINT(118.375480 31.336570)	
+G1791247	Wuhan	POINT(114.266670 30.583330)	
+G1791249	Wuhai	POINT(106.812220 39.664720)	
+G1791388	Wenzhou	POINT(120.666820 27.999420)	
+G1791636	Weinan	POINT(109.508910 34.503550)	
+G1791673	Weihai	POINT(122.113610 37.501670)	
+G1791681	Weifang	POINT(119.101940 36.710000)	
+G1791748	Wanxian	POINT(108.389720 30.803890)	
+G1792260	Wafangdian	POINT(122.008060 39.618330)	
+G1792520	Tongzhou	POINT(116.599440 39.905280)	
+G1792621	Wusong	POINT(117.783330 30.950000)	
+G1792947	Tianjin	POINT(117.176670 39.142220)	
+G1793346	Tangshan	POINT(118.183330 39.633330)	
+G1793424	Tanggu	POINT(117.646940 39.021110)	
+G1793505	Taizhou	POINT(119.910630 32.493330)	
+G1793511	Taiyuan	POINT(112.560280 37.869440)	
+G1793724	Tai\u2019an	POINT(117.120000 36.185280)	
+G1793743	Suzhou	POINT(116.978890 33.636110)	
+G1793879	Suizhou	POINT(113.363060 31.711110)	
+G1794903	Shiyan	POINT(110.778060 32.647500)	
+G1794904	Shiyan	POINT(110.783330 32.566670)	
+G1795060	Shiqi	POINT(113.385210 22.516820)	
+G1795196	Tongchuan	POINT(109.089720 35.080560)	
+G1795270	Shijiazhuang	POINT(114.478610 38.041390)	
+G1795565	Shenzhen	POINT(114.068300 22.545540)	
+G1795816	Shashi	POINT(112.244720 30.307220)	
+G1795855	Shaoxing	POINT(120.581110 30.001670)	
+G1795874	Shaoguan	POINT(113.583330 24.800000)	
+G1795928	Shanwei	POINT(115.347500 22.781990)	
+G1795940	Shantou	POINT(116.714790 23.368140)	
+G1796236	Shanghai	POINT(121.458060 31.222220)	
+G1796663	Sanming	POINT(117.618610 26.248610)	
+G1797121	Jieyang	POINT(116.364160 23.528860)	
+G1797132	Rizhao	POINT(119.455280 35.427500)	
+G1797353	Quanzhou	POINT(118.585830 24.913890)	
+G1797595	Qinhuangdao	POINT(119.588330 39.931670)	
+G1797873	Huaiyin	POINT(119.019170 33.588610)	
+G1797929	Qingdao	POINT(120.371940 36.098610)	
+G1797945	Qingyuan	POINT(113.033330 23.700000)	
+G1798422	Puyang	POINT(115.005280 35.702780)	
+G1798425	Puyang	POINT(119.886110 29.460280)	
+G1798449	Putian	POINT(119.010280 25.439440)	
+G1798654	Pingxiang	POINT(113.850000 27.616670)	
+G1798827	Pingdingshan	POINT(113.301190 33.738470)	
+G1798998	Dadukou	POINT(101.705390 26.547900)	
+G1799397	Ningbo	POINT(121.549450 29.878190)	
+G1799491	Neijiang	POINT(105.062160 29.583540)	
+G1799629	Nanyang	POINT(112.532780 32.994720)	
+G1799722	Nantong	POINT(120.874720 32.030280)	
+G1799846	Nanping	POINT(118.173610 26.645000)	
+G1799869	Nanning	POINT(108.316670 22.816670)	
+G1799962	Nanjing	POINT(118.777780 32.061670)	
+G1800146	Nanchong	POINT(106.084740 30.795080)	
+G1800163	Nanchang	POINT(115.883330 28.683330)	
+G1800627	Mianyang	POINT(104.754240 31.459340)	
+G1800657	Mentougou	POINT(116.091670 39.939170)	
+G1801757	Luqiao	POINT(121.377150 28.580840)	
+G1801792	Luoyang	POINT(112.453610 34.683610)	
+G1801934	Luohe	POINT(114.035280 33.571670)	
+G1802204	Luancheng	POINT(114.651670 37.879170)	
+G1802238	Loudi	POINT(111.994440 27.734440)	
+G1802875	Liuyang	POINT(113.633330 28.150000)	
+G1803318	Linyi	POINT(118.342780 35.063060)	
+G1803331	Linxia	POINT(103.206390 35.600280)	
+G1803422	Linhai	POINT(121.116670 28.850000)	
+G1803567	Linfen	POINT(111.518890 36.088890)	
+G1803791	Licheng	POINT(113.828360 23.295540)	
+G1803834	Liaocheng	POINT(115.964720 36.443890)	
+G1804153	Leshan	POINT(103.763860 29.562280)	
+G1804386	Laohekou	POINT(111.667780 32.385830)	
+G1804430	Lanzhou	POINT(103.792220 36.056390)	
+G1804540	Langfang	POINT(116.694720 39.509720)	
+G1804586	Laiyang	POINT(120.713610 36.975830)	
+G1804651	Kunming	POINT(102.718330 25.038890)	
+G1804850	Kaiyuan	POINT(103.303720 23.697670)	
+G1804879	Kaifeng	POINT(114.348330 34.791110)	
+G1805179	Jiujiang	POINT(115.983330 29.733330)	
+G1805298	Jinzhou	POINT(121.716670 39.100000)	
+G1805518	Jining	POINT(116.581390 35.405000)	
+G1805540	Jingzhou	POINT(112.190280 30.350280)	
+G1805611	Jingmen	POINT(112.204720 31.033610)	
+G1805618	Jingling	POINT(113.100000 30.650000)	
+G1805680	Jingdezhen	POINT(117.207890 29.294700)	
+G1805741	Jincheng	POINT(112.832780 35.502220)	
+G1805753	Jinan	POINT(116.997220 36.668330)	
+G1805953	Jiaxing	POINT(120.748330 30.765560)	
+G1805987	Jiaozuo	POINT(113.233060 35.239720)	
+G1806096	Jiaozhou	POINT(120.003330 36.283890)	
+G1806299	Jiangmen	POINT(113.083330 22.583330)	
+G1806408	Yangjiang	POINT(111.966670 21.850000)	
+G1806445	Ji\u2019an	POINT(114.979270 27.117160)	
+G1806466	Guangyuan	POINT(105.823000 32.442020)	
+G1806535	Huzhou	POINT(120.096390 30.866110)	
+G1806696	Humen	POINT(113.673060 22.818980)	
+G1806776	Huizhou	POINT(114.400000 23.083330)	
+G1806882	Xinhui	POINT(113.048200 22.456000)	
+G1807143	Huangyan	POINT(121.259440 28.647780)	
+G1807234	Huangshi	POINT(115.100000 30.216670)	
+G1807508	Huanggang	POINT(116.999610 23.677040)	
+G1807681	Huainan	POINT(116.996940 32.626390)	
+G1807700	Huaibei	POINT(116.791670 33.974440)	
+G1808198	Heze	POINT(115.441110 35.243060)	
+G1808316	Yiyang	POINT(112.328330 28.589170)	
+G1808370	Hengyang	POINT(112.615000 26.888060)	
+G1808392	Hengshui	POINT(115.701110 37.732220)	
+G1808722	Hefei	POINT(117.280830 31.863890)	
+G1808770	Hebi	POINT(114.192500 35.899170)	
+G1808926	Hangzhou	POINT(120.168890 30.255280)	
+G1808931	Hangu	POINT(117.789170 39.248890)	
+G1808963	Handan	POINT(114.467780 36.600560)	
+G1809061	Jiaojiang	POINT(121.442780 28.680280)	
+G1809078	Haikou	POINT(110.341670 20.045830)	
+G1809412	Guli	POINT(120.033330 28.883330)	
+G1809461	Guiyang	POINT(106.716670 26.583330)	
+G1809498	Guilin	POINT(110.286390 25.281940)	
+G1809858	Guangzhou	POINT(113.250000 23.116670)	
+G1809879	Guangshui	POINT(113.997800 31.619900)	
+G1810295	Gaozhou	POINT(110.846070 21.939240)	
+G1810437	Gaoping	POINT(106.102940 30.775760)	
+G1810458	Gaomi	POINT(119.752780 36.383330)	
+G1810821	Fuzhou	POINT(119.306110 26.061390)	
+G1810845	Fuyang	POINT(115.816670 32.900000)	
+G1810979	Fuling	POINT(107.391940 29.702220)	
+G1811103	Foshan	POINT(113.131480 23.026770)	
+G1811619	Ezhou	POINT(114.833330 30.400000)	
+G1812101	Dongying	POINT(118.485560 37.456390)	
+G1812521	Donghai	POINT(115.642040 22.945940)	
+G1812545	Dongguan	POINT(113.744720 23.048890)	
+G1812728	Dingzhou	POINT(114.995560 38.513060)	
+G1812955	Dezhou	POINT(116.292500 37.448610)	
+G1812961	Deyang	POINT(104.381980 31.130190)	
+G1813253	Dayan	POINT(100.220720 26.868790)	
+G1814082	Daliang	POINT(113.247010 22.850420)	
+G1814087	Dalian	POINT(121.602220 38.912220)	
+G1814757	Chuzhou	POINT(118.297780 32.321940)	
+G1814786	Yangchun	POINT(111.783330 22.166670)	
+G1814906	Chongqing	POINT(106.552780 29.562780)	
+G1815059	Chenzhou	POINT(113.033330 25.800000)	
+G1815286	Chengdu	POINT(104.066670 30.666670)	
+G1815302	Chenghua	POINT(116.770070 23.461320)	
+G1815395	Chaozhou	POINT(116.637860 23.665130)	
+G1815456	Changzhou	POINT(119.966670 31.783330)	
+G1815463	Changzhi	POINT(111.738610 35.208890)	
+G1815577	Changsha	POINT(112.966670 28.200000)	
+G1816080	Cangzhou	POINT(116.866670 38.316670)	
+G1816234	Bozhou	POINT(115.770280 33.877220)	
+G1816265	Boshan	POINT(117.833330 36.483330)	
+G1816440	Bengbu	POINT(117.360830 32.940830)	
+G1816670	Beijing	POINT(116.397230 39.907500)	
+G1816705	Beihai	POINT(109.100000 21.483330)	
+G1816971	Baoding	POINT(115.490280 38.851110)	
+G1817240	Baiyin	POINT(104.208060 36.558330)	
+G1817720	Shangyu	POINT(120.871110 30.015560)	
+G1817968	Anshun	POINT(105.933330 26.250000)	
+G1817993	Anqing	POINT(117.050560 30.509170)	
+G1818116	Anbu	POINT(116.680920 23.448950)	
+G1886760	Suzhou	POINT(120.618060 31.311390)	
+G1915223	Zhongshan	POINT(110.582910 21.322560)	
+G1919014	Lianghu	POINT(120.898450 29.991520)	
+G1927639	Yueyang	POINT(113.091940 29.333330)	
+G2033168	Zhaodong	POINT(125.983330 46.083330)	
+G2033196	Zhangjiakou	POINT(114.879440 40.810000)	
+G2033370	Yingkou	POINT(122.224720 40.668060)	
+G2033413	Yichun	POINT(128.900000 47.700000)	
+G2033467	Yanji	POINT(129.507780 42.907500)	
+G2033574	Xuanhua	POINT(115.044720 40.610280)	
+G2034312	Ulan Hot	POINT(122.083330 46.083330)	
+G2034400	Tongliao	POINT(122.265280 43.612500)	
+G2034439	Tieling	POINT(123.841390 42.293060)	
+G2034655	Suihua	POINT(126.996940 46.640560)	
+G2034714	Siping	POINT(124.368610 43.163330)	
+G2034786	Shuangyashan	POINT(131.153890 46.636110)	
+G2034937	Shenyang	POINT(123.432780 41.792220)	
+G2035225	Ranghulu	POINT(124.866670 46.650000)	
+G2035261	Qitaihe	POINT(130.850000 45.800000)	
+G2035265	Qiqihar	POINT(123.967220 47.340830)	
+G2035513	Panshan	POINT(122.049440 41.188060)	
+G2035644	Nanpiao	POINT(120.747920 41.098220)	
+G2035715	Mudanjiang	POINT(129.600000 44.583330)	
+G2035980	Longfeng	POINT(125.116670 46.550000)	
+G2036109	Liaoyuan	POINT(125.135830 42.903610)	
+G2036113	Liaoyang	POINT(123.173060 41.271940)	
+G2036389	Jixi	POINT(130.966670 45.300000)	
+G2036401	Jiutai	POINT(125.832780 44.152500)	
+G2036427	Jinzhou	POINT(121.141670 41.107780)	
+G2036434	Lianshan	POINT(120.853270 40.764320)	
+G2036458	Jining	POINT(113.105830 41.027500)	
+G2036502	Jilin	POINT(126.560280 43.850830)	
+G2036581	Jiamusi	POINT(130.350000 46.833330)	
+G2036670	Hulan Ergi	POINT(123.633330 47.204170)	
+G2036892	Hohhot	POINT(111.652220 40.810560)	
+G2036920	Hengshan	POINT(130.916670 45.200000)	
+G2036986	Hegang	POINT(130.366670 47.400000)	
+G2037013	Harbin	POINT(126.650000 45.750000)	
+G2037078	Hailar	POINT(119.700000 49.200000)	
+G2037086	Haicheng	POINT(122.741670 40.851940)	
+G2037346	Fuxin	POINT(121.658890 42.015560)	
+G2037355	Fushun	POINT(123.923330 41.855830)	
+G2037620	Dongling	POINT(123.575830 41.814440)	
+G2037799	Datong	POINT(113.291390 40.093610)	
+G2037860	Daqing	POINT(125.000000 46.583330)	
+G2037886	Dandong	POINT(124.394720 40.129170)	
+G2038067	Chifeng	POINT(118.963610 42.268330)	
+G2038087	Chengde	POINT(117.936110 40.972500)	
+G2038120	Chaoyang	POINT(120.458610 41.570280)	
+G2038180	Changchun	POINT(125.322780 43.880000)	
+G2038300	Benxi	POINT(123.765000 41.288610)	
+G2038342	Beipiao	POINT(120.779170 41.791940)	
+G2038432	Baotou	POINT(109.822220 40.652220)	
+G2038569	Baicheng	POINT(122.816670 45.616670)	
+G2038584	Baishan	POINT(126.428610 41.943060)	
+G2038632	Anshan	POINT(122.990000 41.123610)	
+G2038650	Anda	POINT(125.316670 46.400000)	
+G7158935	\u4e1c\u6d77\u5c9b	POINT(110.396130 21.024480)	
+G7283386	Changshu City	POINT(120.742210 31.646150)	
+G7304020	Fenghuang	POINT(109.599610 27.935570)	
+G7602670	Zhu Cheng City	POINT(119.402590 35.995020)	
+G3665900	Villavicencio	POINT(-73.626640 4.142000)	
+G3666304	Valledupar	POINT(-73.250560 10.476940)	
+G3666645	Tulu�	POINT(-76.200000 4.086670)	
+G3667849	Soledad	POINT(-74.766670 10.917220)	
+G3667905	Soacha	POINT(-74.221390 4.587220)	
+G3667983	Sincelejo	POINT(-75.397780 9.304720)	
+G3668605	Santa Marta	POINT(-74.201670 11.247220)	
+G3671916	Popay�n	POINT(-76.613160 2.438230)	
+G3672486	Pereira	POINT(-75.696110 4.813330)	
+G3672778	Pasto	POINT(-77.281110 1.213610)	
+G3673164	Palmira	POINT(-76.303610 3.539440)	
+G3673899	Neiva	POINT(-75.281880 2.927300)	
+G3674453	Monter�a	POINT(-75.890000 8.757500)	
+G3674962	Medell�n	POINT(-75.536110 6.291390)	
+G3675443	Manizales	POINT(-75.520560 5.070000)	
+G3680450	Itag��	POINT(-75.611390 6.171940)	
+G3680656	Ibagu�	POINT(-75.232220 4.438890)	
+G3682385	Floridablanca	POINT(-73.089720 7.064720)	
+G3682631	Envigado	POINT(-75.563890 6.173060)	
+G3685095	Dos Quebradas	POINT(-75.672500 4.834720)	
+G3685533	C�cuta	POINT(-72.505280 7.883330)	
+G3687238	Cartagena	POINT(-75.514440 10.399720)	
+G3687925	Cali	POINT(-76.522500 3.437220)	
+G3688451	Buenaventura	POINT(-77.069720 3.893330)	
+G3688465	Bucaramanga	POINT(-73.125830 7.129720)	
+G3688689	Bogot�	POINT(-74.081750 4.609710)	
+G3688928	Bello	POINT(-75.562220 6.338890)	
+G3689147	Barranquilla	POINT(-74.796390 10.963890)	
+G3689169	Barrancabermeja	POINT(-73.854720 7.065280)	
+G3689560	Armenia	POINT(-75.681110 4.533890)	
+G3621849	San Jos�	POINT(-84.083330 9.933330)	
+G3536729	Santiago de Cuba	POINT(-75.821940 20.024720)	
+G3537906	Santa Clara	POINT(-79.966670 22.400000)	
+G3544091	Pinar del R�o	POINT(-83.698060 22.417500)	
+G3550598	Las Tunas	POINT(-76.951110 20.961670)	
+G3553478	Havana	POINT(-82.383040 23.133020)	
+G3556969	Holgu�n	POINT(-76.263060 20.887220)	
+G3557689	Guant�namo	POINT(-75.209170 20.144440)	
+G3564124	Cienfuegos	POINT(-80.435560 22.146110)	
+G3566067	Camag�ey	POINT(-77.916940 21.380830)	
+G3567597	Bayamo	POINT(-76.643330 20.379170)	
+G146268	Nicosia	POINT(33.366670 35.166670)	
+G146384	Limassol	POINT(33.033330 34.675000)	
+G3067696	Praha	POINT(14.420760 50.088040)	
+G3068160	Plze\u0148	POINT(13.377590 49.747470)	
+G3068799	Ostrava	POINT(18.282040 49.834650)	
+G3078610	Brno	POINT(16.607960 49.195220)	
+G2805753	Wuppertal	POINT(7.183330 51.266670)	
+G2809346	Wiesbaden	POINT(8.250000 50.083330)	
+G2825297	Stuttgart	POINT(9.177020 48.782320)	
+G2831580	Solingen	POINT(7.083330 51.183330)	
+G2842647	Saarbr�cken	POINT(7.000000 49.233330)	
+G2844588	Rostock	POINT(12.140490 54.088700)	
+G2848756	Berlin Reinickendorf	POINT(13.333330 52.566670)	
+G2855598	Berlin Pankow	POINT(13.401860 52.569260)	
+G2856883	Osnabr�ck	POINT(8.050000 52.266670)	
+G2857458	Oldenburg	POINT(8.200000 53.166670)	
+G2860410	Oberhausen	POINT(6.850000 51.466670)	
+G2861650	Nuremberg	POINT(11.068330 49.447780)	
+G2864072	Neue Neustadt	POINT(11.633330 52.150000)	
+G2864118	Neu�	POINT(6.683330 51.200000)	
+G2867543	M�nster	POINT(7.625710 51.962360)	
+G2867714	M�nchen	POINT(11.575490 48.137430)	
+G2867838	M�lheim an der Ruhr	POINT(6.883330 51.433330)	
+G2869894	M�nchengladbach	POINT(6.433330 51.200000)	
+G2873891	Mannheim	POINT(8.464720 49.488330)	
+G2874225	Mainz	POINT(8.271110 50.000000)	
+G2874545	Magdeburg	POINT(11.666670 52.166670)	
+G2875376	Ludwigshafen am Rhein	POINT(8.435280 49.481110)	
+G2875601	L�beck	POINT(10.687290 53.868930)	
+G2878234	Leverkusen	POINT(7.000000 51.033330)	
+G2879139	Leipzig	POINT(12.371290 51.339620)	
+G2884509	Krefeld	POINT(6.566670 51.333330)	
+G2886242	K�ln	POINT(6.950000 50.933330)	
+G2891122	Kiel	POINT(10.134890 54.321330)	
+G2892518	Kassel	POINT(9.500000 51.316670)	
+G2892794	Karlsruhe	POINT(8.385830 49.004720)	
+G2905891	Herne	POINT(7.216670 51.550000)	
+G2910685	Harburg	POINT(9.983330 53.466670)	
+G2910831	Hannover	POINT(9.733220 52.370520)	
+G2911240	Hamm	POINT(7.820890 51.680330)	
+G2911285	Wandsbek	POINT(10.100000 53.566670)	
+G2911287	Marienthal	POINT(10.083330 53.566670)	
+G2911293	Eimsb�ttel	POINT(9.983330 53.566670)	
+G2911296	Altona	POINT(9.933330 53.550000)	
+G2911298	Hamburg	POINT(10.000000 53.550000)	
+G2911522	Halle	POINT(12.000000 51.500000)	
+G2912621	Hagen	POINT(7.466670 51.350000)	
+G2921466	Gelsenkirchen	POINT(7.050000 51.516670)	
+G2925177	Freiburg	POINT(7.852220 47.995900)	
+G2925533	Frankfurt am Main	POINT(8.683330 50.116670)	
+G2928810	Essen	POINT(7.016670 51.450000)	
+G2929670	Erfurt	POINT(11.033330 50.983330)	
+G2934246	D�sseldorf	POINT(6.776160 51.221720)	
+G2934691	Duisburg	POINT(6.750000 51.433330)	
+G2935022	Dresden	POINT(13.738320 51.050890)	
+G2935517	Dortmund	POINT(7.450000 51.516670)	
+G2940132	Chemnitz	POINT(12.916670 50.833330)	
+G2944388	Bremen	POINT(8.807770 53.075160)	
+G2945024	Braunschweig	POINT(10.533330 52.266670)	
+G2946447	Bonn	POINT(7.100000 50.733330)	
+G2947416	Bochum	POINT(7.216670 51.483330)	
+G2949186	Bielefeld	POINT(8.533330 52.033330)	
+G2950159	Berlin	POINT(13.410530 52.524370)	
+G2954172	Augsburg	POINT(10.883330 48.366670)	
+G3247449	Aachen	POINT(6.083420 50.776640)	
+G6545310	Berlin Mitte	POINT(13.404890 52.520030)	
+G6941055	Bochum-Hordel	POINT(7.175600 51.501680)	
+G7289614	Halle Neustadt	POINT(11.916050 51.479240)	
+G7290245	Berlin Steglitz Zehlendorf	POINT(13.241830 52.434850)	
+G7290251	Berlin Wilmersdorf	POINT(13.290970 52.500970)	
+G7290252	Berlin Spandau	POINT(13.199210 52.551100)	
+G223817	Djibouti	POINT(43.144680 11.587670)	
+G2618425	Copenhagen	POINT(12.565530 55.675940)	
+G2624652	�rhus	POINT(10.210760 56.156740)	
+G3492908	Santo Domingo	POINT(-69.988570 18.500120)	
+G3492914	Santiago de los Caballeros	POINT(-70.700000 19.450000)	
+G3493032	San Pedro de Macor�s	POINT(-69.297180 18.461560)	
+G3500957	La Romana	POINT(-68.972850 18.427340)	
+G3511540	San Crist�bal	POINT(-70.100000 18.416670)	
+G2474141	Boumerdas	POINT(3.477170 36.766390)	
+G2479536	Skikda	POINT(6.909210 36.876170)	
+G2481007	Sidi Bel Abb�s	POINT(-0.630850 35.189940)	
+G2485926	Oran	POINT(-0.641670 35.691110)	
+G2498766	El Achir	POINT(4.627440 36.063860)	
+G2498954	Ech Chettia	POINT(1.255380 36.195910)	
+G2501152	Constantine	POINT(6.614720 36.365000)	
+G2505329	Beja�a	POINT(5.084330 36.755870)	
+G2505572	Batna	POINT(6.174140 35.555970)	
+G2505854	Bab Ezzouar	POINT(3.182910 36.726150)	
+G2506999	Annaba	POINT(7.766670 36.900000)	
+G2507480	Algiers	POINT(3.041970 36.752500)	
+G3651297	Santo Domingo de los Colorados	POINT(-79.150000 -0.250000)	
+G3652462	Quito	POINT(-78.524950 -0.229850)	
+G3652941	Portoviejo	POINT(-80.450000 -1.050000)	
+G3654410	Manta	POINT(-80.733330 -0.950000)	
+G3654533	Machala	POINT(-79.966670 -3.266670)	
+G3657509	Guayaquil	POINT(-79.900000 -2.166670)	
+G3658192	Dur�n	POINT(-79.833330 -2.200000)	
+G3658666	Cuenca	POINT(-78.983330 -2.883330)	
+G3660689	Ambato	POINT(-78.616750 -1.249080)	
+G588409	Tallinn	POINT(24.753530 59.436960)	
+G347497	Tanda	POINT(30.998060 30.791110)	
+G347591	\u1e6calkha	POINT(31.373890 31.053060)	
+G347796	S\u016bh\u0101j	POINT(31.700000 26.550000)	
+G350550	Qin\u0101	POINT(32.727220 26.170000)	
+G354775	Kafr ad Daww\u0101r	POINT(30.128430 31.133850)	
+G355795	Hilwan	POINT(31.333330 29.850000)	
+G358448	Damanh�r	POINT(30.466670 31.033330)	
+G358619	Port Said	POINT(32.307500 31.280560)	
+G359173	Ban\u012b Suwayf	POINT(31.088890 29.063890)	
+G359280	Banh\u0101	POINT(31.187500 30.460830)	
+G359493	Az Zaq\u0101z\u012bq	POINT(31.510280 30.591390)	
+G359783	Asy\u016b\u0163	POINT(31.182780 27.182780)	
+G359792	Asw\u0101n	POINT(32.898890 24.087500)	
+G359796	Suez	POINT(32.550000 29.966670)	
+G360502	Luxor	POINT(32.642100 25.698930)	
+G360630	Cairo	POINT(31.249670 30.062630)	
+G360686	Al Miny\u0101	POINT(30.744440 28.119440)	
+G360761	Al Man\u015f\u016brah	POINT(31.376670 31.043060)	
+G360829	Al Ma\u1e29allah al Kubr�	POINT(31.166940 30.976110)	
+G360995	Al J\u012bzah	POINT(31.212220 30.008610)	
+G361055	Ismailia	POINT(32.277220 30.605280)	
+G361058	Alexandria	POINT(29.919170 31.198060)	
+G361320	Al Fayy\u016bm	POINT(30.840000 29.307780)	
+G411165	Idfu	POINT(32.874720 24.980280)	
+G2462881	La�youne / El Aai�n	POINT(-13.203150 27.162240)	
+G343300	Asmara	POINT(38.933330 15.333330)	
+G2509954	Valencia	POINT(-0.377390 39.469750)	
+G2510911	Sevilla	POINT(-5.986940 37.377220)	
+G2511174	Santa Cruz de Tenerife	POINT(-16.254620 28.468240)	
+G2511401	La Laguna	POINT(-16.316670 28.483330)	
+G2512989	Palma	POINT(2.650240 39.569390)	
+G2513416	Murcia	POINT(-1.116670 37.983330)	
+G2514256	M�laga	POINT(-4.420340 36.720160)	
+G2515270	Las Palmas de Gran Canaria	POINT(-15.416670 28.100000)	
+G2516326	Jerez de la Frontera	POINT(-6.133330 36.683330)	
+G2517117	Granada	POINT(-3.606670 37.188170)	
+G2518559	Elx	POINT(-0.701070 38.262180)	
+G2519240	C�rdoba	POINT(-4.766670 37.883330)	
+G2519752	Castell� de la Plana	POINT(-0.033330 39.983330)	
+G2520058	Cartagena	POINT(-0.983330 37.600000)	
+G2521886	Almer�a	POINT(-2.459740 36.838140)	
+G2521978	Alicante	POINT(-0.481490 38.345170)	
+G2522258	Albacete	POINT(-1.850000 38.983330)	
+G3104324	Zaragoza	POINT(-0.877340 41.656060)	
+G3104499	Vitoria-Gasteiz	POINT(-2.666670 42.850000)	
+G3105976	Vigo	POINT(-8.716670 42.233330)	
+G3106672	Valladolid	POINT(-4.716670 41.650000)	
+G3108286	Terrassa	POINT(2.016670 41.566670)	
+G3109718	Santander	POINT(-3.804440 43.464720)	
+G3110044	San Sebasti�n	POINT(-1.974990 43.312830)	
+G3111108	Salamanca	POINT(-5.650000 40.966670)	
+G3111199	Sabadell	POINT(2.109420 41.543290)	
+G3114472	Pamplona	POINT(-1.643230 42.816870)	
+G3114711	Oviedo	POINT(-5.844760 43.360290)	
+G3116025	M�stoles	POINT(-3.864960 40.322340)	
+G3117735	Madrid	POINT(-3.702560 40.416500)	
+G3118150	Logro�o	POINT(-2.450000 42.466670)	
+G3118594	Legan�s	POINT(-3.763500 40.327180)	
+G3119841	A Coru�a	POINT(-8.396000 43.371350)	
+G3120619	L'Hospitalet de Llobregat	POINT(2.100280 41.359670)	
+G3121424	Gij�n	POINT(-5.664440 43.541110)	
+G3121437	Getafe	POINT(-3.732950 40.305710)	
+G3121960	Fuenlabrada	POINT(-3.800000 40.283330)	
+G3127461	Burgos	POINT(-3.700000 42.350000)	
+G3128026	Bilbao	POINT(-2.925280 43.262710)	
+G3128760	Barcelona	POINT(2.158990 41.388790)	
+G3129028	Badalona	POINT(2.247410 41.450040)	
+G3130564	Alcorc�n	POINT(-3.824870 40.345820)	
+G3130616	Alcal� de Henares	POINT(-3.366670 40.483330)	
+G6252065	Nou Barris	POINT(2.177270 41.441630)	
+G6544100	Eixample	POINT(2.161790 41.388960)	
+G6544105	Sant Mart�	POINT(2.199330 41.418140)	
+G330186	Nazr\u0113t	POINT(39.266670 8.550000)	
+G331180	Mek\u2019el\u0113	POINT(39.475280 13.496670)	
+G336014	Gonder	POINT(37.466670 12.600000)	
+G338832	Dir\u0113 Dawa	POINT(41.866110 9.593060)	
+G342884	Bahir Dar	POINT(37.383330 11.600000)	
+G344979	Addis Ababa	POINT(38.746890 9.024970)	
+G632453	Vantaa	POINT(25.040990 60.294140)	
+G633679	Turku	POINT(22.268690 60.451480)	
+G634963	Tampere	POINT(23.787120 61.499110)	
+G658225	Helsinki	POINT(24.935450 60.169520)	
+G660158	Espoo	POINT(24.652200 60.205200)	
+G2972315	Toulouse	POINT(1.443670 43.604260)	
+G2972328	Toulon	POINT(5.933330 43.116670)	
+G2973783	Strasbourg	POINT(7.742960 48.583420)	
+G2980291	Saint-�tienne	POINT(4.400000 45.433330)	
+G2983990	Rennes	POINT(-1.683330 48.083330)	
+G2984114	Reims	POINT(4.033330 49.250000)	
+G2988507	Paris	POINT(2.348800 48.853410)	
+G2990440	Nice	POINT(7.266080 43.703130)	
+G2990969	Nantes	POINT(-1.553360 47.217250)	
+G2992166	Montpellier	POINT(3.883330 43.600000)	
+G2995469	Marseille	POINT(5.381070 43.296950)	
+G2996944	Lyon	POINT(4.850000 45.750000)	
+G2998324	Lille	POINT(3.066670 50.633330)	
+G3003796	Le Havre	POINT(0.107670 49.493800)	
+G3014728	Grenoble	POINT(5.716670 45.166670)	
+G3031582	Bordeaux	POINT(-0.566670 44.833330)	
+G3037656	Angers	POINT(-0.550000 47.466670)	
+G2399697	Libreville	POINT(9.450000 0.383330)	
+G2633691	Wolverhampton	POINT(-2.122960 52.585470)	
+G2634853	Walsall	POINT(-1.983960 52.585280)	
+G2636389	Swindon	POINT(-1.781160 51.557970)	
+G2636432	Swansea	POINT(-3.943230 51.620790)	
+G2636503	Sutton	POINT(-0.200000 51.350000)	
+G2636531	Sunderland	POINT(-1.382220 54.904650)	
+G2636841	Stoke-on-Trent	POINT(-2.185380 53.004150)	
+G2637433	Southend-on-Sea	POINT(0.714330 51.537820)	
+G2637487	Southampton	POINT(-1.404280 50.903950)	
+G2638077	Sheffield	POINT(-1.465900 53.382970)	
+G2639577	Reading	POINT(-0.971130 51.456250)	
+G2639912	Preston	POINT(-2.716670 53.766670)	
+G2639996	Portsmouth	POINT(-1.091250 50.798990)	
+G2640101	Poole	POINT(-2.000000 50.716670)	
+G2640194	Plymouth	POINT(-4.143050 50.371530)	
+G2640729	Oxford	POINT(-1.255960 51.752220)	
+G2641170	Nottingham	POINT(-1.150470 52.953600)	
+G2641181	Norwich	POINT(1.298340 52.627830)	
+G2641430	Northampton	POINT(-0.883330 52.250000)	
+G2641673	Newcastle upon Tyne	POINT(-1.613960 54.973280)	
+G2642465	Milton Keynes	POINT(-0.755830 52.041720)	
+G2643123	Manchester	POINT(-2.237430 53.480950)	
+G2643339	Luton	POINT(-0.417480 51.879670)	
+G2643741	City of London	POINT(-0.091840 51.512790)	
+G2643743	London	POINT(-0.125740 51.508530)	
+G2644210	Liverpool	POINT(-2.977940 53.410580)	
+G2644668	Leicester	POINT(-1.131690 52.638600)	
+G2644688	Leeds	POINT(-1.547850 53.796480)	
+G2645425	Hull	POINT(-0.335250 53.744600)	
+G2646003	Islington	POINT(-0.103040 51.536220)	
+G2648579	Glasgow	POINT(-4.257630 55.865150)	
+G2650225	Edinburgh	POINT(-3.196480 55.952060)	
+G2650752	Dundee	POINT(-2.966670 56.500000)	
+G2650839	Dudley	POINT(-2.083330 52.500000)	
+G2651347	Derby	POINT(-1.476630 52.922770)	
+G2652221	Coventry	POINT(-1.512170 52.406560)	
+G2653822	Cardiff	POINT(-3.180000 51.480000)	
+G2654675	Bristol	POINT(-2.596650 51.455230)	
+G2654993	Bradford	POINT(-1.752060 53.793910)	
+G2655095	Bournemouth	POINT(-1.879500 50.720480)	
+G2655603	Birmingham	POINT(-1.899830 52.481420)	
+G2655984	Belfast	POINT(-5.933330 54.583330)	
+G2657832	Aberdeen	POINT(-2.098140 57.143690)	
+G7535661	London Borough of Harrow	POINT(-0.333330 51.566670)	
+G611717	Tbilisi	POINT(44.833680 41.694110)	
+G613607	K'ut'aisi	POINT(42.699740 42.249610)	
+G2294700	Tema	POINT(-0.016670 5.616670)	
+G2294877	Tamale	POINT(-0.833330 9.400000)	
+G2294915	Takoradi	POINT(-1.750000 4.883330)	
+G2298890	Kumasi	POINT(-1.616670 6.683330)	
+G2306079	Achiaman	POINT(-0.333330 5.700000)	
+G2306104	Accra	POINT(-0.196900 5.556020)	
+G2422465	Conakry	POINT(-13.677290 9.537950)	
+G2422488	Camayenne	POINT(-13.687780 9.535000)	
+G2309527	Malabo	POINT(8.783330 3.750000)	
+G2310046	Bata	POINT(9.750000 1.850000)	
+G255274	Piraeus	POINT(23.637080 37.947450)	
+G255683	P�trai	POINT(21.734440 38.244440)	
+G264371	Athens	POINT(23.716220 37.979450)	
+G734077	Thessalon�ki	POINT(22.943890 40.640280)	
+G3587902	Villa Nueva	POINT(-90.587500 14.526940)	
+G3592519	Mixco	POINT(-90.606390 14.633330)	
+G3598132	Guatemala City	POINT(-90.513270 14.640720)	
+G2374775	Bissau	POINT(-15.583330 11.850000)	
+G3378644	Georgetown	POINT(-58.155270 6.804480)	
+G1819609	Kowloon	POINT(114.183330 22.316670)	
+G1819729	Hong Kong	POINT(114.157690 22.285520)	
+G3600949	Tegucigalpa	POINT(-87.206810 14.081800)	
+G3601782	San Pedro Sula	POINT(-88.033330 15.500000)	
+G3186886	Zagreb	POINT(15.977980 45.814440)	
+G3190261	Split	POINT(16.439150 43.508910)	
+G6618983	Zagreb - Centar	POINT(15.977530 45.813130)	
+G3718426	Port-au-Prince	POINT(-72.335000 18.539170)	
+G3719028	P�tionville	POINT(-72.285280 18.512500)	
+G3726786	Delmas 73	POINT(-72.302780 18.544720)	
+G3727135	Croix des Bouquets	POINT(-72.225000 18.575000)	
+G3728338	Carrefour	POINT(-72.399220 18.541140)	
+G715429	Szeged	POINT(20.148240 46.253000)	
+G717582	Miskolc	POINT(20.783330 48.100000)	
+G721472	Debrecen	POINT(21.633330 47.533330)	
+G3046526	P�cs	POINT(18.233330 46.083330)	
+G3054643	Budapest	POINT(19.039910 47.498010)	
+G6942354	Nagyv�rad	POINT(21.927340 47.060530)	
+G1213614	Sunggal	POINT(98.615100 3.576500)	
+G1214189	Percut	POINT(98.864000 3.625300)	
+G1214191	Perbaungan	POINT(98.956000 3.567900)	
+G1214204	Pematangsiantar	POINT(99.068700 2.959500)	
+G1214520	Medan	POINT(98.666670 3.583330)	
+G1215355	Binjai	POINT(98.485400 3.600100)	
+G1215502	Banda Aceh	POINT(95.322200 5.557700)	
+G1621177	Yogyakarta	POINT(110.360830 -7.782780)	
+G1622786	Makassar	POINT(119.422100 -5.140000)	
+G1624494	Tegal	POINT(109.140200 -6.869400)	
+G1624647	Tasikmalaya	POINT(108.200000 -7.333330)	
+G1624917	Bandarlampung	POINT(105.258030 -5.425440)	
+G1625084	Tangerang	POINT(106.630000 -6.178060)	
+G1625812	Surakarta	POINT(110.831670 -7.556110)	
+G1625822	Surabaya	POINT(112.750830 -7.249170)	
+G1626100	Sumedang Utara	POINT(107.916670 -6.850000)	
+G1626381	Sukabumi	POINT(106.926670 -6.918060)	
+G1626560	Soreang	POINT(107.518330 -7.033060)	
+G1626801	Situbondo	POINT(114.009760 -7.706230)	
+G1627549	Serang	POINT(106.150200 -6.114900)	
+G1627896	Semarang	POINT(110.420300 -6.993200)	
+G1629001	Samarinda	POINT(117.150000 -0.500000)	
+G1629131	Salatiga	POINT(110.492780 -7.331940)	
+G1629710	Rengasdengklok	POINT(107.298060 -6.159170)	
+G1630328	Purwokerto	POINT(109.234440 -7.421390)	
+G1630333	Purwodadi	POINT(110.915800 -7.086800)	
+G1630341	Purwakarta	POINT(107.443330 -6.556940)	
+G1630634	Probolinggo	POINT(113.215900 -7.754300)	
+G1630789	Pontianak	POINT(109.333330 -0.033330)	
+G1630997	Plumbon	POINT(108.472780 -6.705000)	
+G1631648	Pemalang	POINT(109.366670 -6.900000)	
+G1631761	Pekanbaru	POINT(101.450000 0.533330)	
+G1631766	Pekalongan	POINT(109.675300 -6.888600)	
+G1632033	Pasuruan	POINT(112.907500 -7.645300)	
+G1632228	Pasarkemis	POINT(106.530280 -6.170280)	
+G1632276	Parung	POINT(106.733060 -6.421390)	
+G1632937	Pamulang	POINT(106.738330 -6.342780)	
+G1633034	Palu	POINT(119.870700 -0.891700)	
+G1633070	Palembang	POINT(104.745800 -2.916730)	
+G1633419	Padang	POINT(100.354270 -0.949240)	
+G1635882	Mataram	POINT(116.116670 -8.583330)	
+G1636544	Manado	POINT(124.845500 1.487000)	
+G1636556	Mamuju	POINT(118.888500 -2.674800)	
+G1636722	Malang	POINT(112.630400 -7.979700)	
+G1636930	Madiun	POINT(111.523900 -7.629800)	
+G1637510	Loa Janan	POINT(117.095030 -0.582950)	
+G1638063	Lembang	POINT(107.617500 -6.811670)	
+G1638284	Lawang	POINT(112.694700 -7.835300)	
+G1638868	Labuhanbajo	POINT(119.887700 -8.496400)	
+G1640344	Kendari	POINT(122.498890 -3.945000)	
+G1640660	Kediri	POINT(112.016670 -7.816670)	
+G1642588	Jember	POINT(113.703170 -8.166040)	
+G1642858	Jambi	POINT(103.616670 -1.600000)	
+G1642911	Jakarta	POINT(106.845130 -6.214620)	
+G1645524	Depok	POINT(106.818610 -6.400000)	
+G1645528	Denpasar	POINT(115.216670 -8.650000)	
+G1645895	Curug	POINT(106.556390 -6.265830)	
+G1646170	Cirebon	POINT(108.557000 -6.706300)	
+G1646194	Ciputat	POINT(106.695560 -6.237500)	
+G1646448	Cimahi	POINT(107.542500 -6.872220)	
+G1646494	Cileungsi	POINT(106.959170 -6.394720)	
+G1647003	Cibinong	POINT(106.854170 -6.481670)	
+G1647383	Ciampea	POINT(106.700830 -6.554720)	
+G1648473	Bogor	POINT(106.789170 -6.594440)	
+G1649150	Bengkulu	POINT(102.265540 -3.800440)	
+G1649378	Bekasi	POINT(106.989600 -6.234900)	
+G1650213	Banjarmasin	POINT(114.591000 -3.324420)	
+G1650227	Banjaran	POINT(107.587780 -7.045280)	
+G1650357	Bandung	POINT(107.618610 -6.903890)	
+G1650527	Balikpapan	POINT(116.828870 -1.267530)	
+G1651531	Ambon	POINT(128.200000 -3.716670)	
+G1985663	Cikupa	POINT(106.508330 -6.236390)	
+G2057087	Kupang	POINT(123.583330 -10.166670)	
+G2964506	D�n Laoghaire	POINT(-6.135860 53.293950)	
+G2964574	Dublin	POINT(-6.267190 53.343990)	
+G2965140	Cork	POINT(-8.470610 51.897970)	
+G281184	Jerusalem	POINT(35.225300 31.779020)	
+G293397	Tel Aviv	POINT(34.766670 32.066670)	
+G293703	Rishon Le\u1e94iyyon	POINT(34.804440 31.964170)	
+G294071	Netanya	POINT(34.857780 32.333610)	
+G294751	H\u0331olon	POINT(34.772220 32.011390)	
+G294801	Haifa	POINT(34.989170 32.815560)	
+G295530	Beersheba	POINT(34.791300 31.251810)	
+G295629	Ashdod	POINT(34.650000 31.816670)	
+G6693674	Petah Tikva	POINT(34.885030 32.091740)	
+G7498240	West Jerusalem	POINT(35.219610 31.781990)	
+G1252797	Yamun\u0101nagar	POINT(77.283330 30.100000)	
+G1252948	Warangal	POINT(79.583330 18.000000)	
+G1253084	Vizianagaram	POINT(83.416670 18.116670)	
+G1253102	Vish\u0101khapatnam	POINT(83.300000 17.700000)	
+G1253133	Vir\u0101r	POINT(72.800000 19.466670)	
+G1253184	Vijayaw\u0101da	POINT(80.616670 16.516670)	
+G1253237	Ver\u0101val	POINT(70.366670 20.900000)	
+G1253286	Vellore	POINT(79.133330 12.933330)	
+G1253405	Benares	POINT(83.000000 25.333330)	
+G1253573	Vadodara	POINT(73.200000 22.300000)	
+G1253747	Unn\u0101o	POINT(80.500000 26.533330)	
+G1253894	Ulh\u0101snagar	POINT(73.150000 19.216670)	
+G1253914	Ujjain	POINT(75.766670 23.183330)	
+G1253986	Udaipur	POINT(73.691830 24.571170)	
+G1254089	Tumk\u016br	POINT(77.101670 13.342220)	
+G1254163	Thiruvananthapuram	POINT(76.916670 8.483330)	
+G1254187	Trich\u016br	POINT(76.216670 10.516670)	
+G1254241	Tonk	POINT(75.783330 26.166670)	
+G1254348	Tirupp\u016br	POINT(77.350000 11.100000)	
+G1254360	Tirupati	POINT(79.416670 13.650000)	
+G1254361	Tirunelveli	POINT(77.700000 8.733330)	
+G1254388	Tiruchchir\u0101ppalli	POINT(78.683330 10.816670)	
+G1254649	Thanj\u0101v\u016br	POINT(79.150000 10.800000)	
+G1254661	Th\u0101ne	POINT(72.966670 19.200000)	
+G1254745	Teni	POINT(77.483330 10.000000)	
+G1255349	Surendranagar	POINT(71.683330 22.700000)	
+G1255364	S\u016brat	POINT(72.833330 21.166670)	
+G1255634	Sr\u012bnagar	POINT(74.816670 34.083330)	
+G1255744	Son\u012bpat	POINT(77.016670 28.983330)	
+G1255969	S\u012bt\u0101pur	POINT(80.683330 27.566670)	
+G1256052	Sirsa	POINT(75.016670 29.533330)	
+G1256237	Shimla	POINT(77.166670 31.100000)	
+G1256287	Silchar	POINT(92.800000 24.816670)	
+G1256320	S\u012bkar	POINT(75.150000 27.616670)	
+G1256422	Shr\u012br\u0101mpur	POINT(88.342220 22.752780)	
+G1256436	Sol\u0101pur	POINT(75.916670 17.683330)	
+G1256451	Shivpur\u012b	POINT(77.650000 25.433330)	
+G1256515	Shimoga	POINT(75.566670 13.916670)	
+G1256525	Shiliguri	POINT(88.433330 26.700000)	
+G1256728	Sh\u0101hjah\u0101npur	POINT(79.916670 27.883330)	
+G1257022	Satna	POINT(80.833330 24.583330)	
+G1257416	S\u0101ngli	POINT(74.564170 16.854380)	
+G1257540	Sambhal	POINT(78.550000 28.583330)	
+G1257542	Sambalpur	POINT(83.966670 21.450000)	
+G1257629	Salem	POINT(78.166670 11.650000)	
+G1257806	Sah\u0101ranpur	POINT(77.550000 29.966670)	
+G1257845	S\u0101gar	POINT(78.716670 23.833330)	
+G1258076	Rohtak	POINT(76.566670 28.900000)	
+G1258182	Rewa	POINT(81.300000 24.533330)	
+G1258342	Ratl\u0101m	POINT(75.066670 23.316670)	
+G1258526	R\u0101nch\u012b	POINT(85.333330 23.350000)	
+G1258599	R\u0101mpur	POINT(79.033330 28.816670)	
+G1258831	R\u0101j N\u0101ndgaon	POINT(81.033330 21.100000)	
+G1258847	R\u0101jkot	POINT(70.783330 22.300000)	
+G1258932	R\u0101jahmundry	POINT(81.783330 16.983330)	
+G1258980	Raipur	POINT(81.633330 21.233330)	
+G1259004	Raigarh Fort	POINT(73.433330 18.250000)	
+G1259009	R\u0101iganj	POINT(88.116670 25.616670)	
+G1259012	R\u0101ich\u016br	POINT(77.366670 16.200000)	
+G1259064	R\u0101e Bareli	POINT(81.233330 26.216670)	
+G1259091	Quilon	POINT(76.600000 8.883330)	
+G1259166	P\u016brnia	POINT(87.466670 25.783330)	
+G1259184	Puri	POINT(85.850000 19.800000)	
+G1259229	Pune	POINT(73.855350 18.519570)	
+G1259239	Pun\u0101sa	POINT(76.400000 22.233330)	
+G1259312	Proddat\u016br	POINT(78.550000 14.733330)	
+G1259425	Pondicherry	POINT(79.830000 11.930000)	
+G1259652	Pimpri	POINT(73.800000 18.616670)	
+G1260086	Patna	POINT(85.116670 25.600000)	
+G1260107	Pati\u0101la	POINT(76.400280 30.326670)	
+G1260137	Path\u0101nkot	POINT(75.650000 32.283330)	
+G1260341	Parbhani	POINT(76.783330 19.266670)	
+G1260476	P\u0101n\u012bpat	POINT(76.968060 29.388890)	
+G1260482	P\u0101nih\u0101ti	POINT(88.374440 22.694170)	
+G1260692	Pall\u0101varam	POINT(80.183610 12.976110)	
+G1260716	P\u0101li	POINT(73.333330 25.766670)	
+G1261039	Orai	POINT(79.466670 25.983330)	
+G1261045	Ongole	POINT(80.050000 15.500000)	
+G1261258	Niz\u0101m\u0101b\u0101d	POINT(78.116670 18.666670)	
+G1261481	New Delhi	POINT(77.224450 28.635760)	
+G1261529	Nellore	POINT(79.966670 14.433330)	
+G1261731	N\u0101sik	POINT(73.800000 19.983330)	
+G1261913	N\u0101ngloi J\u0101t	POINT(77.066670 28.683330)	
+G1261927	Nandy\u0101l	POINT(78.483330 15.483330)	
+G1262131	Naih\u0101ti	POINT(88.416940 22.902780)	
+G1262180	N\u0101gpur	POINT(79.100000 21.150000)	
+G1262204	N\u0101gercoil	POINT(77.433330 8.166670)	
+G1262292	Nadi\u0101d	POINT(72.866670 22.700000)	
+G1262321	Mysore	POINT(76.649720 12.307220)	
+G1262330	Muzaffarpur	POINT(85.400000 26.116670)	
+G1262332	Muzaffarnagar	POINT(77.683330 29.466670)	
+G1262395	Murw\u0101ra	POINT(80.400000 23.850000)	
+G1262482	Munger	POINT(86.466670 25.383330)	
+G1262771	Morena	POINT(78.000000 26.496940)	
+G1262801	Mor\u0101d\u0101b\u0101d	POINT(78.783330 28.833330)	
+G1262995	Mirz\u0101pur	POINT(82.583330 25.150000)	
+G1263214	Meerut	POINT(77.700000 28.983330)	
+G1263220	Medin\u012bpur	POINT(87.333330 22.433330)	
+G1263311	Mau	POINT(83.550000 25.950000)	
+G1263364	Mathura	POINT(77.683330 27.500000)	
+G1263780	Mangalore	POINT(74.883330 12.866670)	
+G1264115	M\u0101legaon	POINT(74.533330 20.550000)	
+G1264521	Madurai	POINT(78.116670 9.933330)	
+G1264527	Chennai	POINT(80.278470 13.087840)	
+G1264543	Madhyamgram	POINT(88.450000 22.700000)	
+G1264637	Machil\u012bpatnam	POINT(81.133330 16.166670)	
+G1264728	Ludhi\u0101na	POINT(75.850000 30.900000)	
+G1264733	Lucknow	POINT(80.916670 26.850000)	
+G1264773	Loni	POINT(77.283330 28.750000)	
+G1265014	L\u0101t\u016br	POINT(76.583330 18.400000)	
+G1265711	Kulti	POINT(86.850000 23.733330)	
+G1265767	K\u016bkatpalli	POINT(78.416670 17.483330)	
+G1265873	Calicut	POINT(75.766670 11.250000)	
+G1266049	Kota	POINT(75.833330 25.183330)	
+G1266122	Korba	POINT(82.683330 22.350000)	
+G1266285	Kolh\u0101pur	POINT(74.216670 16.700000)	
+G1266976	Kharagpur	POINT(87.333330 22.333330)	
+G1267031	Khandwa	POINT(76.333330 21.833330)	
+G1267076	Khammam	POINT(80.150000 17.250000)	
+G1267480	Katih\u0101r	POINT(87.583330 25.533330)	
+G1267708	Karn\u0101l	POINT(76.983330 29.683330)	
+G1267755	Kar\u012bmnagar	POINT(79.150000 18.433330)	
+G1267995	K\u0101npur	POINT(80.350000 26.466670)	
+G1268159	K\u0101nchipuram	POINT(79.716670 12.833330)	
+G1268257	K\u0101m\u0101rh\u0101ti	POINT(88.374720 22.671110)	
+G1268295	Kaly\u0101n	POINT(73.150000 19.250000)	
+G1268561	K\u0101kin\u0101da	POINT(82.216670 16.933330)	
+G1268773	J\u016bn\u0101gadh	POINT(70.466670 21.516670)	
+G1268782	Jalandhar	POINT(75.579170 31.325560)	
+G1268865	Jodhpur	POINT(73.030000 26.286670)	
+G1268907	J\u012bnd	POINT(76.316670 29.316670)	
+G1269006	Jh\u0101nsi	POINT(78.583330 25.433330)	
+G1269135	Jaunpur	POINT(82.683330 25.733330)	
+G1269280	J\u0101muria	POINT(87.083330 23.700000)	
+G1269300	Jamshedpur	POINT(86.183330 22.800000)	
+G1269317	J\u0101mnagar	POINT(70.066670 22.466670)	
+G1269321	Jammu	POINT(74.866670 32.733330)	
+G1269395	J\u0101lna	POINT(75.883330 19.833330)	
+G1269407	J\u0101lgaon	POINT(75.566670 21.016670)	
+G1269515	Jaipur	POINT(75.816670 26.916670)	
+G1269633	Jabalpur	POINT(79.950060 23.166970)	
+G1269723	Ingr\u0101j B\u0101z\u0101r	POINT(88.150000 25.000000)	
+G1269743	Indore	POINT(75.833300 22.717920)	
+G1269771	Imph\u0101l	POINT(93.950000 24.816670)	
+G1269834	Ichalkaranji	POINT(74.466670 16.700000)	
+G1269843	Hyder\u0101b\u0101d	POINT(78.474440 17.375280)	
+G1269910	Hugli	POINT(88.402500 22.895560)	
+G1269920	Hubli	POINT(75.166670 15.350000)	
+G1269935	Hospet	POINT(76.400000 15.266670)	
+G1269937	Hoshi\u0101rpur	POINT(75.917220 31.532220)	
+G1270022	His\u0101r	POINT(75.716670 29.166670)	
+G1270351	Haridw\u0101r	POINT(78.166670 29.966670)	
+G1270393	H\u0101pur	POINT(77.783330 28.716670)	
+G1270396	H\u0101ora	POINT(88.310280 22.589170)	
+G1270407	Hanum\u0101ngarh	POINT(74.316670 29.583330)	
+G1270583	Gwalior	POINT(78.179170 26.223610)	
+G1270642	Gurgaon	POINT(77.033330 28.466670)	
+G1270668	Gunt\u016br	POINT(80.450000 16.300000)	
+G1270711	Guna	POINT(77.316670 24.650000)	
+G1270752	Gulbarga	POINT(76.833330 17.333330)	
+G1270926	Gorakhpur	POINT(75.683330 29.450000)	
+G1270927	Gorakhpur	POINT(83.373890 26.755000)	
+G1271308	Gh\u0101zi\u0101b\u0101d	POINT(77.433330 28.666670)	
+G1271439	Gaya	POINT(85.000000 24.783330)	
+G1271476	Guw\u0101h\u0101ti	POINT(91.750950 26.186170)	
+G1271685	Gang\u0101nagar	POINT(73.883330 29.916670)	
+G1271715	G\u0101ndh\u012bnagar	POINT(72.683330 23.216670)	
+G1271850	Gadag	POINT(75.616670 15.416670)	
+G1271885	F\u012broz\u0101b\u0101d	POINT(78.416670 27.150000)	
+G1271912	Fatehpur	POINT(80.800000 25.933330)	
+G1271942	Farrukh\u0101b\u0101d	POINT(79.566670 27.400000)	
+G1271951	Far\u012bd\u0101b\u0101d	POINT(77.316670 28.433330)	
+G1271976	Faiz\u0101b\u0101d	POINT(82.133330 26.783330)	
+G1271987	Et\u0101wah	POINT(79.023900 26.776900)	
+G1272051	El\u016bru	POINT(81.100000 16.700000)	
+G1272175	Durg\u0101pur	POINT(87.316670 23.483330)	
+G1272181	Durg	POINT(81.283330 21.183330)	
+G1272423	Dombivli	POINT(73.083330 19.216670)	
+G1272543	Dindigul	POINT(77.950000 10.350000)	
+G1272691	Dhule	POINT(74.783330 20.900000)	
+G1272979	Dhanb\u0101d	POINT(86.450000 23.800000)	
+G1273066	Dew\u0101s	POINT(76.066670 22.966670)	
+G1273294	Delhi	POINT(77.216670 28.666670)	
+G1273313	Dehra D\u016bn	POINT(78.033330 30.316670)	
+G1273491	Darbhanga	POINT(85.900000 26.166670)	
+G1273581	D\u0101n\u0101pur	POINT(85.050000 25.633330)	
+G1273780	Cuttack	POINT(85.879270 20.464970)	
+G1273802	Cuddalore	POINT(79.750000 11.750000)	
+G1273865	Coimbatore	POINT(76.966670 11.000000)	
+G1273874	Cochin	POINT(76.233330 9.966670)	
+G1274693	Chandrapur	POINT(79.300000 19.950000)	
+G1274746	Chand\u012bgarh	POINT(76.793300 30.734300)	
+G1274784	Chandannagar	POINT(88.377220 22.869170)	
+G1275004	Calcutta	POINT(88.369720 22.569720)	
+G1275068	Burh\u0101npur	POINT(76.233330 21.300000)	
+G1275120	Bulandshahr	POINT(77.850000 28.400000)	
+G1275163	Budaun	POINT(79.116670 28.050000)	
+G1275198	Brahmapur	POINT(84.783330 19.316670)	
+G1275248	Borivli	POINT(72.850000 19.233330)	
+G1275339	Mumbai	POINT(72.847940 19.014410)	
+G1275362	Bok\u0101ro	POINT(85.966670 23.783330)	
+G1275637	Bil\u0101spur	POINT(82.150000 22.083330)	
+G1275665	B\u012bk\u0101ner	POINT(73.300000 28.016670)	
+G1275701	Bij\u0101pur	POINT(75.700000 16.833330)	
+G1275716	Bih\u0101r Shar\u012bf	POINT(85.516670 25.183330)	
+G1275738	B\u012bdar	POINT(77.550000 17.900000)	
+G1275778	Bhus\u0101wal	POINT(75.766670 21.050000)	
+G1275817	Bhubaneshwar	POINT(85.833330 20.233330)	
+G1275841	Bhop\u0101l	POINT(77.400000 23.266670)	
+G1275899	Bhiw\u0101ni	POINT(76.133330 28.783330)	
+G1275901	Bhiwandi	POINT(73.066670 19.300000)	
+G1275926	Bhind	POINT(78.788330 26.564170)	
+G1275960	Bh\u012blw\u0101ra	POINT(74.633330 25.350000)	
+G1275971	Bhilai	POINT(81.433330 21.216670)	
+G1276032	Bh\u0101vnagar	POINT(72.150000 21.766670)	
+G1276058	Bh\u0101tp\u0101ra	POINT(88.408890 22.871390)	
+G1276070	Bhatinda	POINT(74.950000 30.200000)	
+G1276100	Bhar\u016bch	POINT(72.966670 21.700000)	
+G1276128	Bharatpur	POINT(77.483330 27.216670)	
+G1276300	Bh\u0101galpur	POINT(87.000000 25.250000)	
+G1276321	Bhadr\u0101vati	POINT(75.716670 13.866670)	
+G1276509	Bellary	POINT(76.933330 15.150000)	
+G1276533	Belgaum	POINT(74.500000 15.866670)	
+G1277013	Bareilly	POINT(79.416670 28.350000)	
+G1277029	Barddham\u0101n	POINT(87.850000 23.250000)	
+G1277065	B\u0101r\u0101sat	POINT(88.516670 22.716670)	
+G1277082	Baranagar	POINT(88.365280 22.643330)	
+G1277333	Bengalore	POINT(77.603290 12.976230)	
+G1277397	B\u0101nda	POINT(80.333330 25.483330)	
+G1277539	B\u0101li	POINT(88.340280 22.646110)	
+G1277799	Bahraich	POINT(81.600000 27.583330)	
+G1277820	Baharampur	POINT(88.250000 24.100000)	
+G1277835	Bah\u0101durgarh	POINT(76.916670 28.683330)	
+G1278130	\u0100vadi	POINT(80.101670 13.115560)	
+G1278149	Aurang\u0101b\u0101d	POINT(75.333330 19.883330)	
+G1278314	\u0100sansol	POINT(86.983330 23.683330)	
+G1278483	Ara	POINT(84.666670 25.566670)	
+G1278672	Anantapur	POINT(77.600000 14.683330)	
+G1278708	Amroha	POINT(78.466670 28.916670)	
+G1278710	Amritsar	POINT(74.865560 31.633060)	
+G1278718	Amr\u0101vati	POINT(77.750000 20.933330)	
+G1278840	Ambatt\u016br	POINT(80.162220 13.098330)	
+G1278903	Amarn\u0101th	POINT(73.166670 19.200000)	
+G1278946	Alwar	POINT(76.600000 27.566670)	
+G1278985	Alleppey	POINT(76.326400 9.490040)	
+G1278994	Allah\u0101b\u0101d	POINT(81.850000 25.450000)	
+G1279017	Al\u012bgarh	POINT(78.083330 27.883330)	
+G1279064	Alandur	POINT(80.206110 13.002500)	
+G1279105	Akola	POINT(77.000000 20.733330)	
+G1279159	Ajmer	POINT(74.633330 26.450000)	
+G1279186	\u0100\u012bzawl	POINT(92.716670 23.733330)	
+G1279228	Ahmadnagar	POINT(74.733330 19.083330)	
+G1279233	Ahmad\u0101b\u0101d	POINT(72.616670 23.033330)	
+G1279259	\u0100gra	POINT(78.016670 27.183330)	
+G1279290	Agartala	POINT(91.275000 23.836390)	
+G1279335	\u0100doni	POINT(77.283330 15.633330)	
+G1344377	Haldia	POINT(88.109750 22.060460)	
+G1348818	Nangi	POINT(88.215280 22.508330)	
+G6943660	Shivaji Nagar	POINT(73.852630 18.530170)	
+G7279746	Noida	POINT(77.330000 28.580000)	
+G7279754	Singrauli	POINT(82.675350 24.199730)	
+G7284820	Jaigaon	POINT(89.375580 26.847660)	
+G7302826	Lal Bahadur Nagar	POINT(78.557570 17.347690)	
+G7302833	Gajuwaka	POINT(83.216670 17.700000)	
+G7302845	Quthbullapur	POINT(78.458180 17.501070)	
+G7302856	Serilingampalle	POINT(78.301960 17.493130)	
+G91597	S\u0101marr\u0101\u2019	POINT(43.875620 34.200960)	
+G94787	Kirkuk	POINT(44.392220 35.468060)	
+G94824	Karbal\u0101\u2019	POINT(44.008470 32.611440)	
+G95446	Arb\u012bl	POINT(44.010620 36.192570)	
+G97990	Ba\u2018q\u016bbah	POINT(44.655450 33.748460)	
+G98182	Baghdad	POINT(44.400880 33.340580)	
+G98463	As Sulaym\u0101n\u012byah	POINT(45.437490 35.561130)	
+G98530	As Sam\u0101wah	POINT(45.294400 31.331980)	
+G98717	Ar Ram\u0101d\u012b	POINT(43.305840 33.422570)	
+G98854	An N\u0101\u015fir\u012byah	POINT(46.261020 31.052050)	
+G98860	An Najaf al Ashraf	POINT(44.339760 31.998540)	
+G99071	Al Maw\u015fil al Jad\u012bdah	POINT(43.097770 36.334640)	
+G99072	Mosul	POINT(43.118890 36.335000)	
+G99131	Al K\u016bt	POINT(45.819020 32.514700)	
+G99347	Al \u1e28illah	POINT(44.435260 32.480140)	
+G99454	Al Fall\u016bjah	POINT(43.779510 33.353800)	
+G99532	Al Ba\u015frah	POINT(47.788850 30.534880)	
+G99608	Al \u2018Am\u0101rah	POINT(47.139600 31.840640)	
+G99762	Ad D\u012bw\u0101n\u012byah	POINT(44.930630 31.990510)	
+G100077	Ab\u016b Ghurayb	POINT(44.185000 33.308330)	
+G388349	Al Ba\u015frat al Qad\u012bmah	POINT(47.814910 30.497210)	
+G14256	\u0100z\u0101dshahr	POINT(48.570730 34.790880)	
+G23814	Kahr\u012bz	POINT(47.055300 34.383800)	
+G32767	Qarchak	POINT(51.568890 35.439440)	
+G111453	Zanj\u0101n	POINT(48.478700 36.673600)	
+G111822	Yazd	POINT(54.367500 31.897220)	
+G112214	Var\u0101m\u012bn	POINT(51.645700 35.324200)	
+G112931	Tehr\u0101n	POINT(51.421510 35.694390)	
+G113646	Tabr\u012bz	POINT(46.291900 38.080000)	
+G114259	S\u012brj\u0101n	POINT(55.681400 29.452000)	
+G115019	Sh\u012br\u0101z	POINT(52.538800 29.603600)	
+G116667	S\u0101veh	POINT(50.356600 35.021300)	
+G116996	Shari-i-Tajan	POINT(53.056520 36.562970)	
+G117392	Saqqez	POINT(46.273500 36.249920)	
+G117574	Sanandaj	POINT(46.992300 35.314400)	
+G118063	Sabzev\u0101r	POINT(57.681910 36.212600)	
+G118743	Rasht	POINT(49.583190 37.280770)	
+G119208	Qom	POINT(50.876400 34.640100)	
+G119505	Qazv\u012bn	POINT(50.004900 36.279700)	
+G121801	Or\u016bm\u012byeh	POINT(45.076050 37.552740)	
+G122285	Neysh\u0101b\u016br	POINT(58.795760 36.213290)	
+G122438	Naz\u0327ar\u0101b\u0101d	POINT(50.607500 35.952100)	
+G124665	Mashhad	POINT(59.606200 36.297000)	
+G125185	Mal\u0101yer	POINT(48.823500 34.296900)	
+G125446	Mah\u0101b\u0101d	POINT(45.722200 36.763100)	
+G126972	Khvoy	POINT(44.952100 38.550300)	
+G127319	Khorramshahr	POINT(48.166400 30.439700)	
+G127349	Khorram\u0101b\u0101d	POINT(48.355830 33.487780)	
+G128226	Kerm\u0101nsh\u0101h	POINT(47.065000 34.314170)	
+G128234	Kerm\u0101n	POINT(57.081230 30.293680)	
+G128477	K\u0101sh\u0101n	POINT(51.436440 33.983080)	
+G128747	Karaj	POINT(51.010300 35.835500)	
+G132144	Hamad\u0101n	POINT(48.514560 34.799220)	
+G132892	Gorg\u0101n	POINT(54.434750 36.838660)	
+G139817	Bandar B\u016bshehr	POINT(50.838500 28.968400)	
+G139889	B\u016bk\u0101n	POINT(46.208900 36.521000)	
+G140044	Bor\u016bjerd	POINT(48.751600 33.897300)	
+G140380	Bojn\u016brd	POINT(57.329030 37.474730)	
+G140463	B\u012brjand	POINT(59.221140 32.866280)	
+G141681	Bandar \u2018Abb\u0101s	POINT(56.280800 27.186500)	
+G142363	B\u0101bol	POINT(52.678950 36.551320)	
+G143083	Ardab\u012bl	POINT(48.293300 38.249800)	
+G143127	Ar\u0101k	POINT(49.689160 34.091740)	
+G143534	\u0100mol	POINT(52.350720 36.469610)	
+G144448	Ahv\u0101z	POINT(48.669300 31.320300)	
+G145459	\u0100b\u0101d\u0101n	POINT(48.304300 30.339200)	
+G418606	Najaf\u0101b\u0101d	POINT(51.366800 32.634400)	
+G418710	Khomeyn\u012b Shahr	POINT(51.521130 32.700180)	
+G418863	E\u015ffah\u0101n	POINT(51.677610 32.657220)	
+G1159301	Z\u0101hed\u0101n	POINT(60.862900 29.496300)	
+G2523630	Reggio di Calabria	POINT(15.661290 38.110470)	
+G2523920	Palermo	POINT(13.359760 38.115820)	
+G2524170	Messina	POINT(15.549690 38.193270)	
+G2525068	Catania	POINT(15.087190 37.502130)	
+G2525473	Cagliari	POINT(9.134620 39.207380)	
+G3164527	Verona	POINT(10.997790 45.434190)	
+G3164603	Venice	POINT(12.326670 45.438610)	
+G3165185	Trieste	POINT(13.780000 45.648610)	
+G3165524	Torino	POINT(7.686820 45.070490)	
+G3165926	Taranto	POINT(17.229720 40.476110)	
+G3169070	Roma	POINT(12.483900 41.894740)	
+G3169921	Prato	POINT(11.090920 43.884250)	
+G3171457	Parma	POINT(10.328980 44.802660)	
+G3171728	Padova	POINT(11.881810 45.415190)	
+G3172394	Napoli	POINT(14.250000 40.833330)	
+G3173331	Modena	POINT(10.925390 44.647830)	
+G3173435	Milano	POINT(9.189510 45.464270)	
+G3173529	Mestre	POINT(12.242500 45.490280)	
+G3174659	Livorno	POINT(10.316000 43.542640)	
+G3176219	Genova	POINT(8.933860 44.406320)	
+G3176885	Foggia	POINT(15.549250 41.460930)	
+G3176959	Florence	POINT(11.250000 43.766670)	
+G3181554	Brescia	POINT(10.227270 45.524780)	
+G3181928	Bologna	POINT(11.338750 44.493810)	
+G3182351	Bari	POINT(16.851180 41.117730)	
+G3489297	New Kingston	POINT(-76.783190 18.007470)	
+G3489854	Kingston	POINT(-76.793580 17.997020)	
+G246013	W\u0101d\u012b as S\u012br	POINT(35.816670 31.950000)	
+G248946	Irbid	POINT(35.850000 32.555560)	
+G250090	Az Zarq\u0101\u2019	POINT(36.087960 32.072750)	
+G250441	Amman	POINT(35.945030 31.955220)	
+G1847963	Atsugi	POINT(139.359720 35.438890)	
+G1847966	Akashi	POINT(134.983330 34.633330)	
+G1848254	Yono	POINT(139.633330 35.883330)	
+G1848313	Yokosuka	POINT(139.667220 35.283610)	
+G1848354	Yokohama-shi	POINT(139.642500 35.447780)	
+G1848373	Yokkaichi	POINT(136.616670 34.966670)	
+G1848522	Yao	POINT(135.600000 34.616670)	
+G1849053	Utsunomiya-shi	POINT(139.883610 36.565830)	
+G1849372	Uji	POINT(135.800000 34.883330)	
+G1849498	Ube	POINT(131.251110 33.943060)	
+G1849796	Tsu-shi	POINT(136.508610 34.730280)	
+G1849814	Toyota	POINT(137.150000 35.083330)	
+G1849837	Toyonaka	POINT(135.469320 34.782440)	
+G1849846	Toyohashi	POINT(137.383330 34.766670)	
+G1849876	Toyama-shi	POINT(137.211390 36.695280)	
+G1849892	Tottori	POINT(134.233330 35.500000)	
+G1850147	Tokyo	POINT(139.581300 35.614880)	
+G1850158	Tokushima-shi	POINT(134.559440 34.065830)	
+G1850181	Tokorozawa	POINT(139.469030 35.799160)	
+G1850692	Nishi-Tokyo-shi	POINT(139.538300 35.725260)	
+G1850910	Takatsuki	POINT(135.616780 34.848330)	
+G1851002	Takasaki	POINT(139.016670 36.333330)	
+G1851012	Takarazuka	POINT(135.356970 34.799360)	
+G1851032	Takaoka	POINT(137.016670 36.750000)	
+G1851100	Takamatsu-shi	POINT(134.043330 34.340280)	
+G1851307	Tachikawa	POINT(139.418060 35.692780)	
+G1851348	Suzuka	POINT(136.583330 34.883330)	
+G1851483	Suita	POINT(135.515670 34.761430)	
+G1851604	S\u014dka	POINT(139.804440 35.820280)	
+G1851717	Shizuoka-shi	POINT(138.383060 34.976940)	
+G1852140	Shinagawa-ku	POINT(139.730170 35.609020)	
+G1852225	Shimonoseki	POINT(130.950000 33.950000)	
+G1852383	Shimminatoch\u014d	POINT(135.200000 34.183330)	
+G1852899	Sasebo	POINT(129.722780 33.159170)	
+G1853195	Sakai	POINT(135.466670 34.583330)	
+G1853295	Sagamihara	POINT(139.354440 35.553060)	
+G1853303	Saga-shi	POINT(130.298800 33.249320)	
+G1853483	Oyama	POINT(139.800000 36.300000)	
+G1853574	\u014ctsu-shi	POINT(135.868330 35.004440)	
+G1853677	\u014cta	POINT(139.366670 36.300000)	
+G1853909	\u014csaka-shi	POINT(135.502180 34.693740)	
+G1854376	Okazaki	POINT(137.166670 34.950000)	
+G1854383	Okayama-shi	POINT(133.935000 34.661670)	
+G1854487	\u014cita-shi	POINT(131.612500 33.238060)	
+G1854703	\u014cgaki	POINT(136.616670 35.350000)	
+G1854747	Odawara	POINT(139.159720 35.255560)	
+G1854902	Numazu	POINT(138.866670 35.100000)	
+G1855207	Nishinomiya	POINT(135.333330 34.716670)	
+G1855431	Niigata-shi	POINT(139.023610 37.902220)	
+G1855503	Nerima	POINT(139.650000 35.733330)	
+G1855612	Nara-shi	POINT(135.804850 34.685050)	
+G1856035	Naha-shi	POINT(127.681110 26.212500)	
+G1856057	Nagoya-shi	POINT(136.906410 35.181470)	
+G1856177	Nagasaki-shi	POINT(129.873610 32.744720)	
+G1856184	Nagareyama	POINT(139.902660 35.856300)	
+G1856199	Nagaoka	POINT(138.850000 37.450000)	
+G1856215	Nagano-shi	POINT(138.181110 36.651390)	
+G1856717	Miyazaki-shi	POINT(131.423890 31.911110)	
+G1856942	Mitaka-shi	POINT(139.559630 35.683510)	
+G1857144	Minami-rinkan	POINT(139.450000 35.483330)	
+G1857519	Matsumoto	POINT(137.966670 36.233330)	
+G1857550	Matsue-shi	POINT(133.050560 35.472220)	
+G1857553	Matsudo	POINT(139.900000 35.783330)	
+G1857843	Maebashi-shi	POINT(139.060830 36.391110)	
+G1857871	Machida	POINT(139.450830 35.540280)	
+G1857910	Kyoto	POINT(135.753850 35.021070)	
+G1858088	Kurume	POINT(130.516670 33.316670)	
+G1858296	Kure	POINT(132.566670 34.233330)	
+G1858311	Kurashiki	POINT(133.766670 34.583330)	
+G1858421	Kumamoto-shi	POINT(130.741670 32.789720)	
+G1858428	Kumagaya	POINT(139.383330 36.133330)	
+G1858729	Koshigaya	POINT(139.783330 35.883330)	
+G1858926	Komaki	POINT(136.916670 35.283330)	
+G1859100	K\u014dfu-shi	POINT(138.568330 35.663890)	
+G1859116	Kodaira	POINT(139.483890 35.726390)	
+G1859146	K\u014dchi-shi	POINT(133.531110 33.559720)	
+G1859171	K\u014dbe-shi	POINT(135.183000 34.691300)	
+G1859307	Kitaky\u016bsh\u016b	POINT(130.833330 33.833330)	
+G1859383	Kishiwada	POINT(135.366670 34.466670)	
+G1859642	Kawasaki	POINT(139.717220 35.520560)	
+G1859675	Kawanishi	POINT(135.416670 34.816670)	
+G1859730	Kawaguchi	POINT(139.720560 35.805000)	
+G1859740	Kawagoe	POINT(139.485280 35.908610)	
+G1859884	Kasukabe	POINT(139.753610 35.976390)	
+G1859891	Kasugai	POINT(136.972290 35.247620)	
+G1859924	Kashiwa	POINT(139.968890 35.854440)	
+G1860243	Kanazawa-shi	POINT(136.625560 36.594440)	
+G1860437	Kami-renjaku	POINT(139.550000 35.683330)	
+G1860672	Kamakura	POINT(139.550280 35.308890)	
+G1860704	Kakogawa	POINT(134.850000 34.766670)	
+G1860827	Kagoshima-shi	POINT(130.558140 31.560180)	
+G1861107	Izumi	POINT(135.433330 34.483330)	
+G1861310	Itami	POINT(135.401260 34.784270)	
+G1861949	Ichinomiya	POINT(136.800000 35.300000)	
+G1861968	Ichikawa	POINT(139.924720 35.719720)	
+G1862033	Ibaraki	POINT(135.568280 34.816410)	
+G1862415	Hiroshima-shi	POINT(132.459370 34.396270)	
+G1862462	Hiratsuka	POINT(139.342220 35.323060)	
+G1862540	Hirakata	POINT(135.649140 34.813520)	
+G1862599	Hino	POINT(139.400280 35.673060)	
+G1862627	Himeji	POINT(134.700000 34.816670)	
+G1863289	Hamamatsu	POINT(137.733330 34.700000)	
+G1863431	Hadano	POINT(139.223610 35.371110)	
+G1863440	Hachi\u014dji	POINT(139.323890 35.655830)	
+G1863641	Gifu-shi	POINT(136.760390 35.422910)	
+G1863905	Funabashi	POINT(139.983330 35.693060)	
+G1863917	Fukuyama	POINT(133.366670 34.483330)	
+G1863967	Fukuoka-shi	POINT(130.418060 33.606390)	
+G1863985	Fukui-shi	POINT(136.222570 36.064430)	
+G1864092	Fujisawa	POINT(139.470000 35.341940)	
+G1864134	Fuji	POINT(138.683330 35.166670)	
+G1864154	Fuch\u016b	POINT(139.483330 35.666670)	
+G1864518	Ch\u014dfu	POINT(139.552220 35.655560)	
+G1864624	Chigasaki	POINT(139.403890 35.326110)	
+G1865005	Ashikaga	POINT(139.450000 36.333330)	
+G1865294	Anj\u014d	POINT(137.080540 34.958280)	
+G1865387	Amagasaki	POINT(135.416670 34.716670)	
+G1865714	Ageo	POINT(139.588610 35.969720)	
+G1907146	Sayama	POINT(139.412120 35.852950)	
+G1926004	Wakayama-shi	POINT(135.167500 34.226110)	
+G1926099	Matsuyama-shi	POINT(132.765740 33.839160)	
+G2110556	Yamagata-shi	POINT(140.363330 38.240560)	
+G2110683	Tsukuba	POINT(140.100000 36.200000)	
+G2111149	Sendai-shi	POINT(140.871940 38.268890)	
+G2111220	Sakura	POINT(140.233330 35.716670)	
+G2111687	Narashino	POINT(140.033330 35.683330)	
+G2111834	Morioka-shi	POINT(141.152500 39.703610)	
+G2111901	Mito-shi	POINT(140.446670 36.341390)	
+G2112141	K\u014driyama	POINT(140.383330 37.400000)	
+G2112312	Katsuta	POINT(140.533330 36.383330)	
+G2112539	Iwaki	POINT(140.883330 37.050000)	
+G2112664	Ichihara	POINT(140.083330 35.516670)	
+G2112708	Hitachi	POINT(140.650000 36.600000)	
+G2112923	Fukushima-shi	POINT(140.467780 37.750000)	
+G2113015	Chiba-shi	POINT(140.123330 35.604720)	
+G2113126	Akita	POINT(140.116670 39.716670)	
+G2113719	Akita-shi	POINT(140.103330 39.718060)	
+G2127733	Tomakomai	POINT(141.603330 42.636940)	
+G2128295	Sapporo-shi	POINT(141.346940 43.064170)	
+G2128815	Obihiro	POINT(143.204440 42.917220)	
+G2129376	Kushiro	POINT(144.374720 42.975000)	
+G2130057	Hirosaki	POINT(140.472500 40.593060)	
+G2130188	Hakodate	POINT(140.736670 41.775830)	
+G2130203	Hachinohe	POINT(141.500000 40.500000)	
+G2130629	Asahikawa	POINT(142.370280 43.767780)	
+G2130658	Aomori-shi	POINT(140.740000 40.824440)	
+G6697563	Neyagawa	POINT(135.627590 34.766150)	
+G6822096	Hitachi-Naka	POINT(140.534790 36.396590)	
+G6825489	J\u014detsu	POINT(138.236420 37.148280)	
+G6940394	Saitama	POINT(139.656570 35.908070)	
+G7279570	Higashimurayama-shi	POINT(139.468520 35.754590)	
+G184622	Nakuru	POINT(36.066670 -0.283330)	
+G184745	Nairobi	POINT(36.816670 -1.283330)	
+G186301	Mombasa	POINT(39.660500 -4.055000)	
+G191245	Kisumu	POINT(34.750000 -0.100000)	
+G198629	Eldoret	POINT(35.269920 0.520360)	
+G1527534	Osh	POINT(72.790000 40.529440)	
+G1528675	Bishkek	POINT(74.590000 42.870000)	
+G1821306	Phnom Penh	POINT(104.916010 11.562450)	
+G1831142	Sihanoukville	POINT(103.529580 10.609320)	
+G1831797	B\u0103td�mb�ng	POINT(103.198220 13.102710)	
+G1866923	W\u014fnsan	POINT(127.443610 39.152780)	
+G1869446	Songnim	POINT(125.645000 38.754170)	
+G1870883	Sariw\u014fn	POINT(125.755830 38.507220)	
+G1871859	Pyongyang	POINT(125.754320 39.033850)	
+G1873757	Namp\u2019o	POINT(125.407780 38.737500)	
+G1876373	Kaes\u014fng	POINT(126.554440 37.970830)	
+G1877030	H\u016dngnam	POINT(127.618610 39.831670)	
+G1877449	Hamh\u016dng	POINT(127.536390 39.918330)	
+G1877615	Haeju	POINT(125.714720 38.040560)	
+G2040893	Sin\u016diju	POINT(124.398060 40.100560)	
+G2043572	Kanggye-si	POINT(126.585230 40.969460)	
+G2044757	Ch\u2019\u014fngjin	POINT(129.775830 41.795560)	
+G1832157	Y\u014fsu	POINT(127.737780 34.744170)	
+G1832847	Yangju	POINT(127.061690 37.833110)	
+G1833105	W\u014fnju	POINT(127.945280 37.351390)	
+G1833747	Ulsan	POINT(129.316670 35.537220)	
+G1833788	\u016cij\u014fngbu	POINT(127.047400 37.741500)	
+G1835235	Taej\u014fn	POINT(127.419720 36.321390)	
+G1835329	Taegu	POINT(128.591110 35.870280)	
+G1835553	Suw\u014fn	POINT(127.008890 37.291110)	
+G1835648	Sunch\u2019\u014fn	POINT(127.489470 34.948080)	
+G1835848	Seoul	POINT(126.977830 37.568260)	
+G1838524	Pusan	POINT(129.040280 35.102780)	
+G1838716	Puch\u2019\u014fn	POINT(126.783060 37.498890)	
+G1839071	P\u2019ohang	POINT(129.365000 36.032220)	
+G1839652	Osan	POINT(127.070560 37.152220)	
+G1841066	Mokp\u2019o	POINT(126.388610 34.793610)	
+G1841245	Masan	POINT(128.572500 35.208060)	
+G1841603	Ky\u014fngju	POINT(129.211670 35.842780)	
+G1841811	Kwangju	POINT(126.915560 35.154720)	
+G1841988	Kuri	POINT(127.139400 37.598600)	
+G1842025	Kunsan	POINT(126.711390 35.978610)	
+G1842225	Kumi	POINT(128.336000 36.113600)	
+G1842485	Goyang	POINT(126.835000 37.656390)	
+G1842943	Kimhae	POINT(128.881110 35.234170)	
+G1843137	Kangn\u016dng	POINT(128.896110 37.755560)	
+G1843491	Iksan	POINT(126.954440 35.943890)	
+G1843564	Inch\u2019\u014fn	POINT(126.731670 37.453610)	
+G1843702	Ich'\u014fn	POINT(127.442500 37.279170)	
+G1843847	Hwaseong	POINT(126.816900 37.206820)	
+G1845136	Ch\u2019unch\u2019\u014fn	POINT(127.734170 37.874720)	
+G1845457	Ch\u014fnju	POINT(127.148890 35.821940)	
+G1845604	Ch\u2019\u014fngju	POINT(127.489720 36.637220)	
+G1845759	Ch\u2019\u014fnan	POINT(127.152200 36.806500)	
+G1846052	Chinju	POINT(128.084720 35.192780)	
+G1846266	Cheju	POINT(126.521940 33.509720)	
+G1846326	Ch\u2019angw\u014fn	POINT(128.681110 35.228060)	
+G1846898	Anyang	POINT(126.926940 37.392500)	
+G1846918	Ansan	POINT(126.821940 37.323610)	
+G1897000	S\u014fngnam	POINT(127.137780 37.438610)	
+G1948005	Kwangmy\u014fng	POINT(126.866390 37.477220)	
+G608668	Oral	POINT(51.366670 51.233330)	
+G609655	Karagandy	POINT(54.866670 50.066670)	
+G610529	Atyrau	POINT(51.883330 47.116670)	
+G610611	Aqt�be	POINT(57.207180 50.279690)	
+G1516905	Taraz	POINT(71.366670 42.900000)	
+G1518262	Temirtau	POINT(72.948330 50.054440)	
+G1518543	Taldyqorghan	POINT(77.916670 45.000000)	
+G1518980	Shymkent	POINT(69.600000 42.300000)	
+G1519422	Semey	POINT(80.227500 50.411110)	
+G1519922	Qyzylorda	POINT(65.509170 44.852780)	
+G1519928	Qostanay	POINT(63.583330 53.166670)	
+G1519942	Qaraghandy	POINT(73.099440 49.798890)	
+G1520172	Petropavlovsk	POINT(69.162780 54.875280)	
+G1520240	Pavlodar	POINT(76.950000 52.300000)	
+G1520316	Ust\u2019-Kamenogorsk	POINT(82.610280 49.978890)	
+G1526273	Astana	POINT(71.445980 51.180100)	
+G1526384	Almaty	POINT(76.950000 43.250000)	
+G1651944	Vientiane	POINT(102.600000 17.966670)	
+G266826	Tripoli	POINT(35.849720 34.436670)	
+G268064	Sidon	POINT(35.368890 33.563060)	
+G268743	Ra\u2019s Bayr\u016bt	POINT(35.483330 33.900000)	
+G276781	Beirut	POINT(35.494420 33.888940)	
+G1234633	Moratuwa	POINT(79.881600 6.773000)	
+G1242833	Jaffna	POINT(80.013700 9.656700)	
+G1246321	Galkissa	POINT(79.863000 6.829300)	
+G1248991	Colombo	POINT(79.847780 6.931940)	
+G2274895	Monrovia	POINT(-10.796900 6.300540)	
+G593116	Vilnius	POINT(25.279800 54.689160)	
+G598098	Klaip\u0117da	POINT(21.117500 55.717220)	
+G598316	Kaunas	POINT(23.900000 54.900000)	
+G456172	Riga	POINT(24.105890 56.946000)	
+G88319	Bangh\u0101z\u012b	POINT(20.066670 32.116670)	
+G89055	Al Bay\u1e11\u0101\u2019	POINT(21.755060 32.762720)	
+G2210221	Tarh\u016bnah	POINT(13.633200 32.435020)	
+G2210247	Tripoli	POINT(13.187460 32.875190)	
+G2214846	Mi\u015fr\u0101tah	POINT(15.092540 32.375350)	
+G2216885	\u1e28\u0101rat az Z\u0101wiyah	POINT(12.715000 32.763060)	
+G2219905	Al Khums	POINT(14.266670 32.650000)	
+G2528910	T�touan	POINT(-5.372420 35.571090)	
+G2530335	Tangier	POINT(-5.813650 35.780580)	
+G2537763	Sal�	POINT(-6.816600 34.038920)	
+G2537881	Safi	POINT(-9.237180 32.299390)	
+G2538475	Rabat	POINT(-6.832550 34.013250)	
+G2540483	Oujda	POINT(-1.907640 34.680520)	
+G2542051	Mohammedia	POINT(-7.394420 33.706590)	
+G2542715	Mekn�s	POINT(-5.547270 33.893520)	
+G2542997	Marrakech	POINT(-8.008280 31.631480)	
+G2544248	Khouribga	POINT(-6.906300 32.881080)	
+G2544571	Kenitra	POINT(-6.580200 34.261010)	
+G2548885	F�s	POINT(-4.999800 34.037150)	
+G2553604	Casablanca	POINT(-7.619160 33.592780)	
+G2555745	Beni Mellal	POINT(-6.349840 32.337250)	
+G2561668	Agadir	POINT(-9.598150 30.420180)	
+G617239	Tiraspol	POINT(29.643330 46.840280)	
+G618426	Chi\u015fin\u0103u	POINT(28.857500 47.005560)	
+G1053384	Toamasina	POINT(49.383330 -18.166670)	
+G1062663	Mahajanga	POINT(46.316670 -15.716670)	
+G1064890	Fianarantsoa	POINT(47.083330 -21.433330)	
+G1070940	Antananarivo	POINT(47.530980 -18.914330)	
+G785842	Skopje	POINT(21.433330 42.000000)	
+G2460596	Bamako	POINT(-8.000000 12.650000)	
+G1293960	Taunggyi	POINT(97.033330 20.783330)	
+G1295765	Akyab	POINT(92.900000 20.150000)	
+G1298824	Rangoon	POINT(96.156110 16.805280)	
+G1300466	Bago	POINT(96.479720 17.336670)	
+G1308465	Mawlamyine	POINT(97.625560 16.491390)	
+G1308522	Monywa	POINT(95.133330 22.116670)	
+G1309611	Myeik	POINT(98.600000 12.433330)	
+G1309793	Meiktila	POINT(95.866670 20.866670)	
+G1311874	Mandalay	POINT(96.083590 21.974730)	
+G1328421	Pathein	POINT(94.733330 16.783330)	
+G6611854	Nay Pyi Taw	POINT(96.129720 19.745000)	
+G2028462	Ulaanbaatar	POINT(106.883240 47.907710)	
+G1821274	Macau	POINT(113.546110 22.200560)	
+G2377450	Nouakchott	POINT(-15.949750 18.100330)	
+G934154	Port Louis	POINT(57.498890 -20.161940)	
+G927967	Lilongwe	POINT(33.783330 -13.983330)	
+G931755	Blantyre	POINT(35.013870 -15.786820)	
+G3514450	Xochimilco	POINT(-99.107500 19.262220)	
+G3514519	Xico	POINT(-98.933330 19.266670)	
+G3514663	Alvaro Obreg�n	POINT(-99.225000 19.373330)	
+G3514670	Villahermosa	POINT(-92.916670 17.983330)	
+G3514674	Gustavo A. Madero	POINT(-99.095830 19.478610)	
+G3514783	Veracruz	POINT(-96.133330 19.200000)	
+G3515001	Tuxtla Guti�rrez	POINT(-93.116670 16.750000)	
+G3515302	Toluca	POINT(-99.667220 19.288330)	
+G3515428	Tlalpan	POINT(-99.166670 19.283330)	
+G3515431	Tlalnepantla	POINT(-99.221670 19.526940)	
+G3515463	Tlahuac	POINT(-99.003330 19.281670)	
+G3515807	Cuautitl�n Izcalli	POINT(-99.246670 19.646940)	
+G3516109	Tehuac�n	POINT(-97.383330 18.450000)	
+G3516266	Tapachula	POINT(-92.283330 14.900000)	
+G3516355	Tampico	POINT(-97.850000 22.216670)	
+G3518135	San Pablo de las Salinas	POINT(-99.096390 19.665830)	
+G3520339	Reynosa	POINT(-98.283330 26.083330)	
+G3521081	Puebla de Zaragoza	POINT(-98.200000 19.050000)	
+G3521168	Poza Rica de Hidalgo	POINT(-97.459460 20.533150)	
+G3522210	Pachuca de Soto	POINT(-98.733290 20.116970)	
+G3522507	Oaxaca de Ju�rez	POINT(-96.716670 17.050000)	
+G3522551	Nuevo Laredo	POINT(-99.516670 27.500000)	
+G3522732	Nicol�s Romero	POINT(-99.313060 19.621940)	
+G3522790	Naucalpan de Ju�rez	POINT(-99.239630 19.478510)	
+G3523183	Minatitl�n	POINT(-94.516670 17.983330)	
+G3523303	Metepec	POINT(-99.607780 19.253610)	
+G3523349	M�rida	POINT(-89.616670 20.966670)	
+G3523466	Heroica Matamoros	POINT(-97.500000 25.883330)	
+G3523760	Magdalena Contreras	POINT(-99.233330 19.283330)	
+G3523908	Los Reyes	POINT(-98.966670 19.350000)	
+G3526485	Jiutepec	POINT(-99.183330 18.866670)	
+G3526617	Jalapa Enr�quez	POINT(-96.916670 19.533330)	
+G3526682	Ixtapaluca	POINT(-98.883330 19.316670)	
+G3526683	Iztapalapa	POINT(-99.051940 19.351110)	
+G3526700	Iztacalco	POINT(-99.084720 19.396670)	
+G3529612	Ecatepec	POINT(-99.052500 19.601110)	
+G3529947	Cuernavaca	POINT(-99.250000 18.916670)	
+G3530049	Cuajimalpa	POINT(-99.301110 19.355830)	
+G3530139	Coyoac�n	POINT(-99.160280 19.328890)	
+G3530517	Coatzacoalcos	POINT(-94.416670 18.150000)	
+G3530569	Coacalco	POINT(-99.110280 19.631670)	
+G3530580	Ciudad Victoria	POINT(-99.133330 23.733330)	
+G3530589	Ciudad Nezahualc�yotl	POINT(-99.033060 19.413610)	
+G3530594	Ciudad Madero	POINT(-97.833330 22.266670)	
+G3530597	Mexico City	POINT(-99.127660 19.428470)	
+G3530757	Cholula	POINT(-98.303520 19.064060)	
+G3530870	Chilpancingo de los Bravos	POINT(-99.500000 17.550000)	
+G3531200	Chalco de D�az Covarrubias	POINT(-98.900000 19.266670)	
+G3531673	Canc�n	POINT(-86.846560 21.174290)	
+G3531732	Campeche	POINT(-90.533330 19.850000)	
+G3532497	Azcapotzalco	POINT(-99.183610 19.488890)	
+G3532624	Ciudad L�pez Mateos	POINT(-99.261390 19.558330)	
+G3533462	Acapulco de Ju�rez	POINT(-99.890100 16.863360)	
+G3827406	Benito Juarez	POINT(-99.165000 19.385000)	
+G3827407	Venustiano Carranza	POINT(-99.099170 19.430000)	
+G3827408	Miguel Hidalgo	POINT(-99.202780 19.422500)	
+G3827409	Cuauht�moc	POINT(-99.156940 19.417220)	
+G3979770	Zapopan	POINT(-103.400000 20.716670)	
+G3980760	Uruapan del Progreso	POINT(-102.066670 19.416670)	
+G3981254	Torre�n	POINT(-103.433330 25.550000)	
+G3981369	Tonal�	POINT(-103.233330 20.616670)	
+G3981461	Tlaquepaque	POINT(-103.316670 20.650000)	
+G3981609	Tijuana	POINT(-117.016670 32.533330)	
+G3981941	Tepic	POINT(-104.900000 21.500000)	
+G3982912	Soledad D�ez Guti�rrez	POINT(-100.950000 22.200000)	
+G3984583	Santa Catarina	POINT(-100.458130 25.673250)	
+G3985241	San Nicol�s de los Garza	POINT(-100.300000 25.750000)	
+G3985606	San Luis Potos�	POINT(-100.983330 22.150000)	
+G3988086	Saltillo	POINT(-101.000000 25.416670)	
+G3991164	Quer�taro	POINT(-100.383330 20.600000)	
+G3991328	Puerto Vallarta	POINT(-105.230660 20.620410)	
+G3995402	Morelia	POINT(-101.184430 19.700780)	
+G3995465	Monterrey	POINT(-100.316670 25.666670)	
+G3995523	Monclova	POINT(-101.416670 26.900000)	
+G3996069	Mexicali	POINT(-115.468330 32.651940)	
+G3996322	Mazatl�n	POINT(-106.416670 23.216670)	
+G3997479	Los Mochis	POINT(-108.966670 25.766670)	
+G3998655	Le�n	POINT(-101.666670 21.116670)	
+G4000900	La Paz	POINT(-110.300000 24.166670)	
+G4004330	Irapuato	POINT(-101.350000 20.683330)	
+G4004886	Heroica Nogales	POINT(-110.933330 31.333330)	
+G4004898	Hermosillo	POINT(-110.966670 29.066670)	
+G4005492	Guadalupe	POINT(-100.250000 25.683330)	
+G4005539	Guadalajara	POINT(-103.333330 20.666670)	
+G4005775	G�mez Palacio	POINT(-103.500000 25.566670)	
+G4005867	General Escobedo	POINT(-100.333330 25.816670)	
+G4006702	Ensenada	POINT(-116.616670 31.866670)	
+G4011743	Durango	POINT(-104.666670 24.033330)	
+G4012176	Culiac�n	POINT(-107.389720 24.799440)	
+G4013704	Ciudad Obreg�n	POINT(-109.933330 27.483330)	
+G4013708	Ciudad Ju�rez	POINT(-106.483330 31.733330)	
+G4014338	Chihuahua	POINT(-106.083330 28.633330)	
+G4014875	Celaya	POINT(-100.816670 20.516670)	
+G4018390	Apodaca	POINT(-100.200000 25.766670)	
+G4019233	Aguascalientes	POINT(-102.300000 21.883330)	
+G7280708	Colonia del Valle	POINT(-99.162040 19.386110)	
+G1732687	Batu Pahat	POINT(102.933330 1.850000)	
+G1732745	Sekudai	POINT(103.666670 1.533330)	
+G1732752	Johor Bahru	POINT(103.757800 1.465500)	
+G1732811	Kluang	POINT(103.332800 2.025100)	
+G1732903	Shah Alam	POINT(101.532810 3.085070)	
+G1732905	Klang	POINT(101.450000 3.033330)	
+G1733432	Kota Kinabalu	POINT(116.066670 5.983330)	
+G1734052	Sandakan	POINT(118.117900 5.840200)	
+G1734199	Tawau	POINT(117.900000 4.250000)	
+G1734393	Kulim	POINT(100.561770 5.364990)	
+G1734586	Taiping	POINT(100.733330 4.850000)	
+G1734634	Ipoh	POINT(101.082900 4.584100)	
+G1734705	Kuala Terengganu	POINT(103.140800 5.330200)	
+G1734759	Melaka	POINT(102.248060 2.196940)	
+G1734810	Seremban	POINT(101.933330 2.716670)	
+G1735079	Bukit Mertajam	POINT(100.466700 5.363010)	
+G1735106	George Town	POINT(100.335430 5.411230)	
+G1735158	Petaling Jaya	POINT(101.606710 3.107260)	
+G1735161	Kuala Lumpur	POINT(101.686530 3.141200)	
+G1735227	Kuantan	POINT(103.326000 3.807700)	
+G1735498	Sungai Petani	POINT(100.487720 5.647000)	
+G1735634	Kuching	POINT(110.333330 1.550000)	
+G1735902	Sibu	POINT(111.816670 2.300000)	
+G1736309	Alor Setar	POINT(100.360140 6.121040)	
+G1736376	Kota Bharu	POINT(102.238600 6.133280)	
+G1737486	Bintulu	POINT(113.033330 3.166670)	
+G1738050	Miri	POINT(113.983330 4.383330)	
+G1771023	Kampung Baru Subang	POINT(101.533330 3.150000)	
+G1028434	Quelimane	POINT(36.888330 -17.878610)	
+G1033356	Nampula	POINT(39.266600 -15.116460)	
+G1035025	Cidade de Nacala	POINT(40.672780 -14.542780)	
+G1039854	Matola	POINT(32.458890 -25.962220)	
+G1040652	Maputo	POINT(32.589170 -25.965280)	
+G1049261	Chimoio	POINT(33.483330 -19.116390)	
+G1052373	Beira	POINT(34.838890 -19.843610)	
+G3352136	Windhoek	POINT(17.083230 -22.559410)	
+G2437798	Zinder	POINT(8.988370 13.804870)	
+G2440485	Niamey	POINT(2.111780 13.512500)	
+G2441291	Maradi	POINT(7.101740 13.500000)	
+G2317765	Zaria	POINT(7.725180 11.113240)	
+G2319133	Warri	POINT(5.750000 5.516670)	
+G2320576	Umuahia	POINT(7.489590 5.526270)	
+G2320831	Ugep	POINT(8.081200 5.808600)	
+G2322794	Abuja	POINT(7.180830 9.175830)	
+G2322911	Sokoto	POINT(5.239020 13.060920)	
+G2323390	Saki	POINT(3.383330 8.666670)	
+G2323411	Shagamu	POINT(3.647760 6.843230)	
+G2323675	Sapele	POINT(5.676660 5.894050)	
+G2324774	Port Harcourt	POINT(7.013400 4.777420)	
+G2325200	Oyo	POINT(3.933330 7.850000)	
+G2325314	Owo	POINT(5.586810 7.196200)	
+G2325330	Owerri	POINT(7.030410 5.483330)	
+G2326016	Onitsha	POINT(6.788450 6.145430)	
+G2326

<TRUNCATED>

[03/26] lucenenet git commit: Some tidy up

Posted by ni...@apache.org.
Some tidy up


Project: http://git-wip-us.apache.org/repos/asf/lucenenet/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucenenet/commit/9e566a89
Tree: http://git-wip-us.apache.org/repos/asf/lucenenet/tree/9e566a89
Diff: http://git-wip-us.apache.org/repos/asf/lucenenet/diff/9e566a89

Branch: refs/heads/master
Commit: 9e566a89279be1897772c4fbee9dbb3e86e1331a
Parents: 4a485ee
Author: Naz Junaid <na...@gmail.com>
Authored: Wed Jul 27 23:05:18 2016 +0100
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Tue Nov 15 19:17:54 2016 +0700

----------------------------------------------------------------------
 src/Lucene.Net.Spatial/DisjointSpatialFilter.cs |   2 +-
 .../Lucene.Net.Spatial.csproj                   |   9 +-
 .../Prefix/AbstractPrefixTreeFilter.cs          |  52 +++----
 .../Prefix/AbstractVisitingPrefixTreeFilter.cs  |  70 ++++-----
 .../Prefix/ContainsPrefixTreeFilter.cs          |  82 ++++-------
 .../Prefix/IntersectsPrefixTreeFilter.cs        |  29 +---
 .../Prefix/PointPrefixTreeFieldCacheProvider.cs |   1 -
 .../Prefix/PrefixTreeStrategy.cs                |   6 +-
 .../Prefix/RecursivePrefixTreeStrategy.cs       |   6 +-
 .../Prefix/TermQueryPrefixTreeStrategy.cs       |   4 +-
 .../Prefix/WithinPrefixTreeFilter.cs            |  73 +++++-----
 src/Lucene.Net.Spatial/Queries/SpatialArgs.cs   | 142 ------------------
 .../Queries/SpatialArgsParser.cs                | 140 ------------------
 .../Queries/SpatialOperation.cs                 | 116 ---------------
 .../Queries/UnsupportedSpatialOperation.cs      |  30 ----
 src/Lucene.Net.Spatial/Query/SpatialArgs.cs     | 136 ++++++++++++++++++
 .../Query/SpatialArgsParser.cs                  | 144 +++++++++++++++++++
 .../Query/SpatialOperation.cs                   | 114 +++++++++++++++
 .../Query/UnsupportedSpatialOperation.cs        |  30 ++++
 .../Serialized/SerializedDVStrategy.cs          |  69 +++++++++
 src/Lucene.Net.Spatial/SpatialStrategy.cs       |   2 +-
 .../Vector/PointVectorStrategy.cs               |   2 +-
 22 files changed, 627 insertions(+), 632 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs b/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
index e36a97b..3ad309e 100644
--- a/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
+++ b/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
@@ -18,7 +18,7 @@ using System;
 using Lucene.Net.Index;
 using Lucene.Net.Queries;
 using Lucene.Net.Search;
-using Lucene.Net.Spatial.Queries;
+using Lucene.Net.Spatial.Query;
 using Lucene.Net.Util;
 
 namespace Lucene.Net.Spatial

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj b/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj
index fabf96c..f6669e6 100644
--- a/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj
+++ b/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj
@@ -82,10 +82,11 @@
     <Compile Include="Prefix\Tree\SpatialPrefixTreeFactory.cs" />
     <Compile Include="Prefix\WithinPrefixTreeFilter.cs" />
     <Compile Include="Properties\AssemblyInfo.cs" />
-    <Compile Include="Queries\SpatialArgs.cs" />
-    <Compile Include="Queries\SpatialArgsParser.cs" />
-    <Compile Include="Queries\SpatialOperation.cs" />
-    <Compile Include="Queries\UnsupportedSpatialOperation.cs" />
+    <Compile Include="Query\SpatialArgs.cs" />
+    <Compile Include="Query\SpatialArgsParser.cs" />
+    <Compile Include="Query\SpatialOperation.cs" />
+    <Compile Include="Query\UnsupportedSpatialOperation.cs" />
+    <Compile Include="Serialized\SerializedDVStrategy.cs" />
     <Compile Include="SpatialStrategy.cs" />
     <Compile Include="Util\ShapeFieldCacheDistanceValueSource.cs" />
     <Compile Include="Util\CachingDoubleValueSource.cs" />

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
index c70a7c5..fadab36 100644
--- a/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
@@ -23,21 +23,19 @@ using Spatial4n.Core.Shapes;
 
 namespace Lucene.Net.Spatial.Prefix
 {
-    /// <summary>Base class for Lucene Filters on SpatialPrefixTree fields.</summary>
-    /// <remarks>Base class for Lucene Filters on SpatialPrefixTree fields.</remarks>
-    /// <lucene.experimental></lucene.experimental>
+    /// <summary>
+    /// Base class for Lucene Filters on SpatialPrefixTree fields.
+    /// @lucene.experimental
+    /// </summary>
     public abstract class AbstractPrefixTreeFilter : Filter
     {
-        protected internal readonly int detailLevel;
-        protected internal readonly string fieldName;
-
-        protected internal readonly SpatialPrefixTree grid;
         protected internal readonly Shape queryShape;
-
-        public AbstractPrefixTreeFilter(Shape queryShape, string
-                                                              fieldName, SpatialPrefixTree grid, int detailLevel)
+        protected internal readonly string fieldName;
+        protected internal readonly SpatialPrefixTree grid;//not in equals/hashCode since it's implied for a specific field
+        protected internal readonly int detailLevel;
+        
+        public AbstractPrefixTreeFilter(Shape queryShape, string fieldName, SpatialPrefixTree grid, int detailLevel)
         {
-            //not in equals/hashCode since it's implied for a specific field
             this.queryShape = queryShape;
             this.fieldName = fieldName;
             this.grid = grid;
@@ -82,32 +80,29 @@ namespace Lucene.Net.Spatial.Prefix
 
         /// <summary>
         /// Holds transient state and docid collecting utility methods as part of
-        /// traversing a
-        /// <see cref="TermsEnum">Lucene.Net.Index.TermsEnum</see>
-        /// .
+        /// traversing a <see cref="TermsEnum">Lucene.Net.Index.TermsEnum</see>.
         /// </summary>
         public abstract class BaseTermsEnumTraverser
         {
-            private readonly AbstractPrefixTreeFilter _enclosing;
-            protected internal readonly AtomicReaderContext context;
-
-            protected internal readonly int maxDoc;
-            protected internal Bits acceptDocs;
+            private readonly AbstractPrefixTreeFilter outerInstance;
+            protected readonly AtomicReaderContext context;
+            protected Bits acceptDocs;
+            protected readonly int maxDoc;
 
-            protected internal DocsEnum docsEnum;
-            protected internal TermsEnum termsEnum;
+            protected TermsEnum termsEnum;//remember to check for null in getDocIdSet
+            protected DocsEnum docsEnum;
+            
 
             /// <exception cref="System.IO.IOException"></exception>
-            public BaseTermsEnumTraverser(AbstractPrefixTreeFilter _enclosing, AtomicReaderContext
-                                                                                   context, Bits acceptDocs)
+            public BaseTermsEnumTraverser(AbstractPrefixTreeFilter outerInstance, AtomicReaderContext context, Bits acceptDocs)
             {
-                this._enclosing = _enclosing;
-                //remember to check for null in getDocIdSet
+                this.outerInstance = outerInstance;
+                
                 this.context = context;
                 AtomicReader reader = context.AtomicReader;
                 this.acceptDocs = acceptDocs;
                 maxDoc = reader.MaxDoc;
-                Terms terms = reader.Terms(this._enclosing.fieldName);
+                Terms terms = reader.Terms(this.outerInstance.fieldName);
                 if (terms != null)
                 {
                     termsEnum = terms.Iterator(null);
@@ -115,12 +110,11 @@ namespace Lucene.Net.Spatial.Prefix
             }
 
             /// <exception cref="System.IO.IOException"></exception>
-            protected internal virtual void CollectDocs(FixedBitSet bitSet)
+            protected virtual void CollectDocs(FixedBitSet bitSet)
             {
                 //WARN: keep this specialization in sync
                 Debug.Assert(termsEnum != null);
-                docsEnum = termsEnum.Docs(acceptDocs, docsEnum, DocsEnum.FLAG_NONE
-                    );
+                docsEnum = termsEnum.Docs(acceptDocs, docsEnum, DocsEnum.FLAG_NONE);
                 int docid;
                 while ((docid = docsEnum.NextDoc()) != DocIdSetIterator.NO_MORE_DOCS)
                 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
index 980f4cc..861c0de 100644
--- a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
@@ -28,36 +28,27 @@ using Spatial4n.Core.Shapes;
 namespace Lucene.Net.Spatial.Prefix
 {
     /// <summary>
-    /// Traverses a
-    /// <see cref="Lucene.Net.Spatial.Prefix.Tree.SpatialPrefixTree">Lucene.Net.Spatial.Prefix.Tree.SpatialPrefixTree
-    /// 	</see>
-    /// indexed field, using the template &
+    /// Traverses a <see cref="SpatialPrefixTree">SpatialPrefixTree</see> indexed field, using the template &
     /// visitor design patterns for subclasses to guide the traversal and collect
     /// matching documents.
     /// <p/>
-    /// Subclasses implement
-    /// <see cref="Lucene.Net.Search.Filter.GetDocIdSet(AtomicReaderContext, Bits)
-    /// 	">Lucene.Search.Filter.GetDocIdSet(AtomicReaderContext, Bits)
-    /// 	</see>
-    /// by instantiating a custom
-    /// <see cref="VisitorTemplate">VisitorTemplate</see>
+    /// Subclasses implement <see cref="Filter.GetDocIdSet(AtomicReaderContext, Bits)">Lucene.Search.Filter.GetDocIdSet(AtomicReaderContext, Bits)</see>
+    /// by instantiating a custom <see cref="VisitorTemplate">VisitorTemplate</see>
     /// subclass (i.e. an anonymous inner class) and implement the
     /// required methods.
+    /// @lucene.internal
     /// </summary>
-    /// <lucene.internal></lucene.internal>
     public abstract class AbstractVisitingPrefixTreeFilter : AbstractPrefixTreeFilter
     {
+        // Historical note: this code resulted from a refactoring of RecursivePrefixTreeFilter,
+        // which in turn came out of SOLR-2155
+
         protected internal readonly int prefixGridScanLevel;
 
-        public AbstractVisitingPrefixTreeFilter(Shape queryShape
-                                                , string fieldName, SpatialPrefixTree grid, int detailLevel,
-                                                int prefixGridScanLevel
-            )
+        public AbstractVisitingPrefixTreeFilter(Shape queryShape, string fieldName, SpatialPrefixTree grid, 
+                                                int detailLevel, int prefixGridScanLevel)
             : base(queryShape, fieldName, grid, detailLevel)
         {
-            //Historical note: this code resulted from a refactoring of RecursivePrefixTreeFilter,
-            // which in turn came out of SOLR-2155
-            //at least one less than grid.getMaxLevels()
             this.prefixGridScanLevel = Math.Max(0, Math.Min(prefixGridScanLevel, grid.MaxLevels - 1));
             Debug.Assert(detailLevel <= grid.MaxLevels);
         }
@@ -66,9 +57,9 @@ namespace Lucene.Net.Spatial.Prefix
         {
             if (!base.Equals(o))
             {
-                return false;
+                return false;//checks getClass == o.getClass & instanceof
             }
-            //checks getClass == o.getClass & instanceof
+            
             var that = (AbstractVisitingPrefixTreeFilter)o;
             if (prefixGridScanLevel != that.prefixGridScanLevel)
             {
@@ -168,27 +159,20 @@ namespace Lucene.Net.Spatial.Prefix
         /// <lucene.internal></lucene.internal>
         public abstract class VisitorTemplate : BaseTermsEnumTraverser
         {
-            private readonly AbstractVisitingPrefixTreeFilter _enclosing;
+            private readonly AbstractVisitingPrefixTreeFilter outerInstance;
             private readonly BytesRef curVNodeTerm = new BytesRef();
-            protected internal readonly bool hasIndexedLeaves;
-
-            private VNode curVNode;
-
-            private Cell scanCell;
+            protected internal readonly bool hasIndexedLeaves;//if false then we can skip looking for them
 
-            private BytesRef thisTerm;
+            private VNode curVNode;//current pointer, derived from query shape
+            private BytesRef thisTerm; //the result of termsEnum.term()
+            private Cell scanCell;//curVNode.cell's term.
 
             /// <exception cref="System.IO.IOException"></exception>
-            public VisitorTemplate(AbstractVisitingPrefixTreeFilter _enclosing, AtomicReaderContext
-                                                                                    context, Bits acceptDocs,
+            public VisitorTemplate(AbstractVisitingPrefixTreeFilter outerInstance, AtomicReaderContext context, Bits acceptDocs,
                                    bool hasIndexedLeaves)
-                : base(_enclosing, context, acceptDocs)
+                : base(outerInstance, context, acceptDocs)
             {
-                this._enclosing = _enclosing;
-                //if false then we can skip looking for them
-                //current pointer, derived from query shape
-                //curVNode.cell's term.
-                //the result of termsEnum.term()
+                this.outerInstance = outerInstance;
                 this.hasIndexedLeaves = hasIndexedLeaves;
             }
 
@@ -207,7 +191,7 @@ namespace Lucene.Net.Spatial.Prefix
                 }
                 // all done
                 curVNode = new VNode(null);
-                curVNode.Reset(_enclosing.grid.WorldCell);
+                curVNode.Reset(outerInstance.grid.WorldCell);
                 Start();
                 AddIntersectingChildren();
                 while (thisTerm != null)
@@ -257,7 +241,7 @@ namespace Lucene.Net.Spatial.Prefix
                     if (compare > 0)
                     {
                         // leap frog (termsEnum is beyond where we would otherwise seek)
-                        Debug.Assert(!((AtomicReader)context.Reader).Terms(_enclosing.fieldName).Iterator(null).SeekExact(curVNodeTerm), "should be absent");
+                        Debug.Assert(!((AtomicReader)context.Reader).Terms(outerInstance.fieldName).Iterator(null).SeekExact(curVNodeTerm), "should be absent");
                     }
                     else
                     {
@@ -309,7 +293,7 @@ namespace Lucene.Net.Spatial.Prefix
             {
                 Debug.Assert(thisTerm != null);
                 Cell cell = curVNode.cell;
-                if (cell.Level >= _enclosing.detailLevel)
+                if (cell.Level >= outerInstance.detailLevel)
                 {
                     throw new InvalidOperationException("Spatial logic error");
                 }
@@ -320,7 +304,7 @@ namespace Lucene.Net.Spatial.Prefix
                     // then add all of those docs
                     Debug.Assert(StringHelper.StartsWith(thisTerm, curVNodeTerm
                                      ));
-                    scanCell = _enclosing.grid.GetCell(thisTerm.Bytes, thisTerm.Offset
+                    scanCell = outerInstance.grid.GetCell(thisTerm.Bytes, thisTerm.Offset
                                                        , thisTerm.Length, scanCell);
                     if (scanCell.Level == cell.Level && scanCell.IsLeaf())
                     {
@@ -337,7 +321,7 @@ namespace Lucene.Net.Spatial.Prefix
                 // scan through terms beneath this cell.
                 // Scanning is a performance optimization trade-off.
                 //TODO use termsEnum.docFreq() as heuristic
-                bool scan = cell.Level >= _enclosing.prefixGridScanLevel;
+                bool scan = cell.Level >= outerInstance.prefixGridScanLevel;
                 //simple heuristic
                 if (!scan)
                 {
@@ -354,7 +338,7 @@ namespace Lucene.Net.Spatial.Prefix
                 else
                 {
                     //Scan (loop of termsEnum.next())
-                    Scan(_enclosing.detailLevel);
+                    Scan(outerInstance.detailLevel);
                 }
             }
 
@@ -370,7 +354,7 @@ namespace Lucene.Net.Spatial.Prefix
             /// </summary>
             protected internal virtual IEnumerator<Cell> FindSubCellsToVisit(Cell cell)
             {
-                return cell.GetSubCells(_enclosing.queryShape).GetEnumerator();
+                return cell.GetSubCells(outerInstance.queryShape).GetEnumerator();
             }
 
             /// <summary>
@@ -393,7 +377,7 @@ namespace Lucene.Net.Spatial.Prefix
                                             );
                     thisTerm = termsEnum.Next())
                 {
-                    scanCell = _enclosing.grid.GetCell(thisTerm.Bytes, thisTerm.Offset
+                    scanCell = outerInstance.grid.GetCell(thisTerm.Bytes, thisTerm.Offset
                                                        , thisTerm.Length, scanCell);
                     int termLevel = scanCell.Level;
                     if (termLevel > scanDetailLevel)

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
index 23fb602..c6c448f 100644
--- a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
@@ -16,13 +16,13 @@
  */
 using System;
 using System.Collections.Generic;
-using Lucene.Net.Spatial.Queries;
 using Lucene.Net.Spatial.Util;
 using Spatial4n.Core.Shapes;
 using Lucene.Net.Index;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix;
 using Lucene.Net.Spatial.Prefix.Tree;
+using Lucene.Net.Spatial.Query;
 using Lucene.Net.Util;
 
 namespace Lucene.Net.Spatial.Prefix
@@ -59,14 +59,13 @@ namespace Lucene.Net.Spatial.Prefix
             private SpatialPrefixTree grid;
 
             /// <exception cref="System.IO.IOException"></exception>
-            public ContainsVisitor(ContainsPrefixTreeFilter enclosing, AtomicReaderContext context
-                , Bits acceptDocs)
-                : base(enclosing, context, acceptDocs)
+            public ContainsVisitor(ContainsPrefixTreeFilter outerInstance, AtomicReaderContext context, Bits acceptDocs)
+                : base(outerInstance, context, acceptDocs)
             {
-                this.queryShape = enclosing.queryShape;
-                this.detailLevel = enclosing.detailLevel;
-                this.grid = enclosing.grid;
-                this.multiOverlappingIndexedShapes = enclosing.multiOverlappingIndexedShapes;
+                this.queryShape = outerInstance.queryShape;
+                this.detailLevel = outerInstance.detailLevel;
+                this.grid = outerInstance.grid;
+                this.multiOverlappingIndexedShapes = outerInstance.multiOverlappingIndexedShapes;
             }
 
             internal BytesRef termBytes = new BytesRef();
@@ -221,6 +220,11 @@ namespace Lucene.Net.Spatial.Prefix
                 intSet = new SentinelIntSet(size, -1);
             }
 
+            public bool Get(int index)
+            {
+                return intSet.Exists(index);
+            }
+
             public virtual void Set(int index)
             {
                 intSet.Put(index);
@@ -230,14 +234,9 @@ namespace Lucene.Net.Spatial.Prefix
                 }
             }
 
-            /// <summary>Largest docid.</summary>
-            /// <remarks>Largest docid.</remarks>
-            public int Length
+            int Bits.Length()
             {
-                get
-                {
-                    return maxInt;
-                }
+                return maxInt;
             }
 
             /// <summary>Number of docids.</summary>
@@ -248,11 +247,10 @@ namespace Lucene.Net.Spatial.Prefix
             }
 
             /// <summary>NOTE: modifies and returns either "this" or "other"</summary>
-            public virtual ContainsPrefixTreeFilter.SmallDocSet Union(ContainsPrefixTreeFilter.SmallDocSet
-                 other)
+            public virtual SmallDocSet Union(SmallDocSet other)
             {
-                ContainsPrefixTreeFilter.SmallDocSet bigger;
-                ContainsPrefixTreeFilter.SmallDocSet smaller;
+                SmallDocSet bigger;
+                SmallDocSet smaller;
                 if (other.intSet.Size() > this.intSet.Size())
                 {
                     bigger = other;
@@ -275,19 +273,18 @@ namespace Lucene.Net.Spatial.Prefix
                 return bigger;
             }
 
-            /// <exception cref="System.IO.IOException"></exception>
-            public Lucene.Net.Util.Bits Bits
+            public override Bits GetBits()
             {
-                get
-                {
-                    //if the # of docids is super small, return null since iteration is going
-                    // to be faster
-                    return Size() > 4 ? this : null;
-                }
+                //if the # of docids is super small, return null since iteration is going
+                // to be faster
+                return Size() > 4 ? this : null;
             }
 
             private sealed class _DocIdSetIterator_225 : DocIdSetIterator
             {
+                private readonly int size;
+                private readonly int[] docs;
+
                 public _DocIdSetIterator_225(int size, int[] docs)
                 {
                     this.size = size;
@@ -299,9 +296,9 @@ namespace Lucene.Net.Spatial.Prefix
 
                 public override int DocID()
                 {
-                    if (this.idx >= 0 && this.idx < size)
+                    if (idx >= 0 && idx < size)
                     {
-                        return docs[this.idx];
+                        return docs[idx];
                     }
                     else
                     {
@@ -312,11 +309,11 @@ namespace Lucene.Net.Spatial.Prefix
                 /// <exception cref="System.IO.IOException"></exception>
                 public override int NextDoc()
                 {
-                    if (++this.idx < size)
+                    if (++idx < size)
                     {
-                        return docs[this.idx];
+                        return docs[idx];
                     }
-                    return DocIdSetIterator.NO_MORE_DOCS;
+                    return NO_MORE_DOCS;
                 }
 
                 /// <exception cref="System.IO.IOException"></exception>
@@ -324,22 +321,13 @@ namespace Lucene.Net.Spatial.Prefix
                 {
                     //for this small set this is likely faster vs. a binary search
                     // into the sorted array
-                    return this.SlowAdvance(target);
+                    return SlowAdvance(target);
                 }
 
                 public override long Cost()
                 {
                     return size;
                 }
-
-                private readonly int size;
-
-                private readonly int[] docs;
-            }
-            //class SmallDocSet
-            public bool this[int index]
-            {
-                get { return intSet.Exists(index); }
             }
 
             public override DocIdSetIterator GetIterator()
@@ -365,16 +353,6 @@ namespace Lucene.Net.Spatial.Prefix
                 Array.Sort(docs, 0, size);
                 return new _DocIdSetIterator_225(size, docs);
             }
-
-            public bool Get(int index)
-            {
-                throw new NotImplementedException();
-            }
-
-            int Bits.Length()
-            {
-                throw new NotImplementedException();
-            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
index f5d8713..4b8f25b 100644
--- a/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
@@ -49,22 +49,20 @@ namespace Lucene.Net.Spatial.Prefix
         /// <exception cref="System.IO.IOException"></exception>
         public override DocIdSet GetDocIdSet(AtomicReaderContext context, Bits acceptDocs)
         {
-            return new _VisitorTemplate_55(this, context, acceptDocs, hasIndexedLeaves).GetDocIdSet
-                ();
+            return new _VisitorTemplate_55(this, context, acceptDocs, hasIndexedLeaves).GetDocIdSet();
         }
 
         #region Nested type: _VisitorTemplate_55
 
         private sealed class _VisitorTemplate_55 : VisitorTemplate
         {
-            private readonly IntersectsPrefixTreeFilter _enclosing;
+            private readonly IntersectsPrefixTreeFilter outerInstance;
             private FixedBitSet results;
 
-            public _VisitorTemplate_55(IntersectsPrefixTreeFilter _enclosing, AtomicReaderContext baseArg1, 
-                Bits baseArg2, bool baseArg3)
-                : base(_enclosing, baseArg1, baseArg2, baseArg3)
+            public _VisitorTemplate_55(IntersectsPrefixTreeFilter outerInstance, AtomicReaderContext context, Bits acceptDocs, bool hasIndexedLeaves)
+                : base(outerInstance, context, acceptDocs, hasIndexedLeaves)
             {
-                this._enclosing = _enclosing;
+                this.outerInstance = outerInstance;
             }
 
             protected internal override void Start()
@@ -80,8 +78,7 @@ namespace Lucene.Net.Spatial.Prefix
             /// <exception cref="System.IO.IOException"></exception>
             protected internal override bool Visit(Cell cell)
             {
-                if (cell.GetShapeRel() == SpatialRelation.WITHIN || cell.Level == _enclosing
-                                                                                      .detailLevel)
+                if (cell.GetShapeRel() == SpatialRelation.WITHIN || cell.Level == outerInstance.detailLevel)
                 {
                     CollectDocs(results);
                     return false;
@@ -98,19 +95,7 @@ namespace Lucene.Net.Spatial.Prefix
             /// <exception cref="System.IO.IOException"></exception>
             protected internal override void VisitScanned(Cell cell)
             {
-                Shape cShape;
-                //if this cell represents a point, use the cell center vs the box
-                // TODO this behavior is debatable; might want to be configurable
-                // (points never have isLeaf())
-                if (cell.Level == _enclosing.grid.MaxLevels && !cell.IsLeaf())
-                {
-                    cShape = cell.GetCenter();
-                }
-                else
-                {
-                    cShape = cell.GetShape();
-                }
-                if (_enclosing.queryShape.Relate(cShape).Intersects())
+                if (outerInstance.queryShape.Relate(cell.GetShape()).Intersects())
                 {
                     CollectDocs(results);
                 }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs b/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
index 73c57f9..3459cb7 100644
--- a/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
+++ b/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
@@ -45,7 +45,6 @@ namespace Lucene.Net.Spatial.Prefix
 
         private Cell scanCell = null;//re-used in readShape to save GC
 
-        //re-used in readShape to save GC
         protected internal override Point ReadShape(BytesRef term)
         {
             scanCell = grid.GetCell(term.Bytes, term.Offset, term.Length, scanCell);

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
index ff83b83..5f7c406 100644
--- a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
@@ -26,8 +26,8 @@ using Lucene.Net.Analysis.Tokenattributes;
 using Lucene.Net.Documents;
 using Lucene.Net.Index;
 using Lucene.Net.Spatial.Prefix.Tree;
-using Lucene.Net.Spatial.Queries;
 using Lucene.Net.Queries.Function;
+using Lucene.Net.Spatial.Query;
 using Lucene.Net.Spatial.Util;
 using Lucene.Net.Support;
 using Spatial4n.Core.Shapes;
@@ -96,9 +96,7 @@ namespace Lucene.Net.Spatial.Prefix
             new ConcurrentDictionary<string, PointPrefixTreeFieldCacheProvider>();
 
         protected internal readonly bool simplifyIndexedCells;
-
         protected internal int defaultFieldValuesArrayLen = 2;
-
         protected internal double distErrPct = SpatialArgs.DEFAULT_DISTERRPCT;
 
         public PrefixTreeStrategy(SpatialPrefixTree grid, string fieldName, bool simplifyIndexedCells)
@@ -158,8 +156,10 @@ namespace Lucene.Net.Spatial.Prefix
         {
             int detailLevel = grid.GetLevelForDistance(distErr);
             IList<Cell> cells = grid.GetCells(shape, detailLevel, true, simplifyIndexedCells);
+
             //TODO is CellTokenStream supposed to be re-used somehow? see Uwe's comments:
             //  http://code.google.com/p/lucene-spatial-playground/issues/detail?id=4
+
             Field field = new Field(FieldName, new CellTokenStream(cells.GetEnumerator()), FieldType);
             return new Field[] { field };
         }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
index 2fcfe01..8740476 100644
--- a/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
@@ -18,7 +18,7 @@
 using Lucene.Net.Queries.Function;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix.Tree;
-using Lucene.Net.Spatial.Queries;
+using Lucene.Net.Spatial.Query;
 using Spatial4n.Core.Shapes;
 
 namespace Lucene.Net.Spatial.Prefix
@@ -37,9 +37,8 @@ namespace Lucene.Net.Spatial.Prefix
     /// <lucene.experimental></lucene.experimental>
     public class RecursivePrefixTreeStrategy : PrefixTreeStrategy
     {
-        protected readonly SpatialPrefixTree grid;
         private int prefixGridScanLevel;
-
+        
         /** True if only indexed points shall be supported.  See
         *  {@link IntersectsPrefixTreeFilter#hasIndexedLeaves}. */
         protected bool pointsOnly = false;
@@ -50,7 +49,6 @@ namespace Lucene.Net.Spatial.Prefix
         public RecursivePrefixTreeStrategy(SpatialPrefixTree grid, string fieldName)
             : base(grid, fieldName, true) //simplify indexed cells
         {
-            this.grid = grid;
             prefixGridScanLevel = grid.MaxLevels - 4;//TODO this default constant is dependent on the prefix grid size
         }
 

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
index b8753e3..aa67eeb 100644
--- a/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
@@ -18,7 +18,7 @@ using System.Collections.Generic;
 using Lucene.Net.Queries;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix.Tree;
-using Lucene.Net.Spatial.Queries;
+using Lucene.Net.Spatial.Query;
 using Lucene.Net.Util;
 using Spatial4n.Core.Shapes;
 
@@ -65,7 +65,7 @@ namespace Lucene.Net.Spatial.Prefix
             int i = 0;
             foreach (Cell cell in cells)
             {
-                terms[i++] = new BytesRef(cell.TokenString);
+                terms[i++] = new BytesRef(cell.TokenString);//TODO use cell.getTokenBytes()
             }
             return new TermsFilter(FieldName, terms);
         }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
index 07f8f9e..62d5104 100644
--- a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
@@ -20,7 +20,7 @@ using System.Diagnostics;
 using Lucene.Net.Index;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix.Tree;
-using Lucene.Net.Spatial.Queries;
+using Lucene.Net.Spatial.Query;
 using Lucene.Net.Util;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Distance;
@@ -48,7 +48,11 @@ namespace Lucene.Net.Spatial.Prefix
     /// <lucene.experimental></lucene.experimental>
     public class WithinPrefixTreeFilter : AbstractVisitingPrefixTreeFilter
     {
-        private readonly Shape bufferedQueryShape;
+        /// TODO LUCENE-4869: implement faster algorithm based on filtering out false-positives of a
+        //  minimal query buffer by looking in a DocValues cache holding a representative
+        //  point of each disjoint component of a document's shape(s).
+
+        private readonly Shape bufferedQueryShape;//if null then the whole world
 
         /// <summary>
         /// See
@@ -60,16 +64,10 @@ namespace Lucene.Net.Spatial.Prefix
         /// where non-matching documents are looked for so they can be excluded. If
         /// -1 is used then the whole world is examined (a good default for correctness).
         /// </summary>
-        public WithinPrefixTreeFilter(Shape queryShape, string fieldName
-                                      , SpatialPrefixTree grid, int detailLevel, int prefixGridScanLevel,
-                                      double queryBuffer
-            )
+        public WithinPrefixTreeFilter(Shape queryShape, string fieldName, SpatialPrefixTree grid, 
+                                      int detailLevel, int prefixGridScanLevel, double queryBuffer)
             : base(queryShape, fieldName, grid, detailLevel, prefixGridScanLevel)
         {
-            //TODO LUCENE-4869: implement faster algorithm based on filtering out false-positives of a
-            //  minimal query buffer by looking in a DocValues cache holding a representative
-            //  point of each disjoint component of a document's shape(s).
-            //if null then the whole world
             if (queryBuffer == -1)
             {
                 bufferedQueryShape = null;
@@ -80,10 +78,10 @@ namespace Lucene.Net.Spatial.Prefix
             }
         }
 
-        /// <summary>Returns a new shape that is larger than shape by at distErr.</summary>
-        /// <remarks>Returns a new shape that is larger than shape by at distErr.</remarks>
-        protected internal virtual Shape BufferShape(Shape
-                                                         shape, double distErr)
+        /// <summary>
+        /// Returns a new shape that is larger than shape by at distErr
+        /// </summary>
+        protected virtual Shape BufferShape(Shape shape, double distErr)
         {
             //TODO move this generic code elsewhere?  Spatial4j?
             if (distErr <= 0)
@@ -149,8 +147,7 @@ namespace Lucene.Net.Spatial.Prefix
         }
 
         /// <exception cref="System.IO.IOException"></exception>
-        public override DocIdSet GetDocIdSet(AtomicReaderContext context, Bits acceptDocs
-            )
+        public override DocIdSet GetDocIdSet(AtomicReaderContext context, Bits acceptDocs)
         {
             return new _VisitorTemplate_121(this, context, acceptDocs, true).GetDocIdSet();
         }
@@ -159,18 +156,18 @@ namespace Lucene.Net.Spatial.Prefix
 
         private sealed class _VisitorTemplate_121 : VisitorTemplate
         {
-            private readonly WithinPrefixTreeFilter _enclosing;
+            private readonly WithinPrefixTreeFilter outerInstance;
             private FixedBitSet inside;
 
             private FixedBitSet outside;
 
             private SpatialRelation visitRelation;
 
-            public _VisitorTemplate_121(WithinPrefixTreeFilter _enclosing, AtomicReaderContext
-                                                                               baseArg1, Bits baseArg2, bool baseArg3)
-                : base(_enclosing, baseArg1, baseArg2, baseArg3)
+            public _VisitorTemplate_121(WithinPrefixTreeFilter outerInstance, AtomicReaderContext context, 
+                Bits acceptDocs, bool hasIndexedLeaves)
+                : base(outerInstance, context, acceptDocs, hasIndexedLeaves)
             {
-                this._enclosing = _enclosing;
+                this.outerInstance = outerInstance;
             }
 
             protected internal override void Start()
@@ -188,7 +185,7 @@ namespace Lucene.Net.Spatial.Prefix
             protected internal override IEnumerator<Cell> FindSubCellsToVisit(Cell cell)
             {
                 //use buffered query shape instead of orig.  Works with null too.
-                return cell.GetSubCells(_enclosing.bufferedQueryShape).GetEnumerator();
+                return cell.GetSubCells(outerInstance.bufferedQueryShape).GetEnumerator();
             }
 
             /// <exception cref="System.IO.IOException"></exception>
@@ -196,27 +193,21 @@ namespace Lucene.Net.Spatial.Prefix
             {
                 //cell.relate is based on the bufferedQueryShape; we need to examine what
                 // the relation is against the queryShape
-                visitRelation = cell.GetShape().Relate(_enclosing.queryShape);
+                visitRelation = cell.GetShape().Relate(outerInstance.queryShape);
                 if (visitRelation == SpatialRelation.WITHIN)
                 {
                     CollectDocs(inside);
                     return false;
                 }
-                else
+                else if (visitRelation == SpatialRelation.DISJOINT)
                 {
-                    if (visitRelation == SpatialRelation.DISJOINT)
-                    {
-                        CollectDocs(outside);
-                        return false;
-                    }
-                    else
-                    {
-                        if (cell.Level == _enclosing.detailLevel)
-                        {
-                            CollectDocs(inside);
-                            return false;
-                        }
-                    }
+                    CollectDocs(outside);
+                    return false;
+                }
+                else if (cell.Level == outerInstance.detailLevel)
+                {
+                    CollectDocs(inside);
+                    return false;
                 }
                 return true;
             }
@@ -225,8 +216,8 @@ namespace Lucene.Net.Spatial.Prefix
             protected internal override void VisitLeaf(Cell cell)
             {
                 //visitRelation is declared as a field, populated by visit() so we don't recompute it
-                Debug.Assert(_enclosing.detailLevel != cell.Level);
-                Debug.Assert(visitRelation == cell.GetShape().Relate(_enclosing.queryShape));
+                Debug.Assert(outerInstance.detailLevel != cell.Level);
+                Debug.Assert(visitRelation == cell.GetShape().Relate(outerInstance.queryShape));
                 if (AllCellsIntersectQuery(cell, visitRelation))
                 {
                     CollectDocs(inside);
@@ -249,9 +240,9 @@ namespace Lucene.Net.Spatial.Prefix
             {
                 if (relate == SpatialRelation.NULL_VALUE)
                 {
-                    relate = cell.GetShape().Relate(_enclosing.queryShape);
+                    relate = cell.GetShape().Relate(outerInstance.queryShape);
                 }
-                if (cell.Level == _enclosing.detailLevel)
+                if (cell.Level == outerInstance.detailLevel)
                 {
                     return relate.Intersects();
                 }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/Queries/SpatialArgs.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Queries/SpatialArgs.cs b/src/Lucene.Net.Spatial/Queries/SpatialArgs.cs
deleted file mode 100644
index 0383508..0000000
--- a/src/Lucene.Net.Spatial/Queries/SpatialArgs.cs
+++ /dev/null
@@ -1,142 +0,0 @@
-\ufeff/*
- * 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 System.Text;
-using Spatial4n.Core.Context;
-using Spatial4n.Core.Exceptions;
-using Spatial4n.Core.Shapes;
-
-namespace Spatial4n.Core.Exceptions
-{
-    [Serializable]
-    public class InvalidSpatialArgument : ArgumentException
-    {
-        public InvalidSpatialArgument(String reason)
-            : base(reason)
-        {
-        }
-    }
-}
-
-namespace Lucene.Net.Spatial.Queries
-{
-    public class SpatialArgs
-    {
-        public static readonly double DEFAULT_DISTERRPCT = 0.025d;
-
-        public SpatialOperation Operation { get; set; }
-
-        public SpatialArgs(SpatialOperation operation, Shape shape)
-        {
-            if (operation == null || shape == null)
-                throw new ArgumentException("operation and shape are required");
-            this.Operation = operation;
-            this.Shape = shape;
-        }
-
-        /// <summary>
-        /// Computes the distance given a shape and the {@code distErrPct}.  The
-        /// algorithm is the fraction of the distance from the center of the query
-        /// shape to its furthest bounding box corner.
-        /// </summary>
-        /// <param name="shape">Mandatory.</param>
-        /// <param name="distErrPct">0 to 0.5</param>
-        /// <param name="ctx">Mandatory</param>
-        /// <returns>A distance (in degrees).</returns>
-        public static double CalcDistanceFromErrPct(Shape shape, double distErrPct, SpatialContext ctx)
-        {
-            if (distErrPct < 0 || distErrPct > 0.5)
-            {
-                throw new ArgumentException("distErrPct " + distErrPct + " must be between [0 to 0.5]", "distErrPct");
-            }
-            if (distErrPct == 0 || shape is Point)
-            {
-                return 0;
-            }
-            Rectangle bbox = shape.GetBoundingBox();
-            //The diagonal distance should be the same computed from any opposite corner,
-            // and this is the longest distance that might be occurring within the shape.
-            double diagonalDist = ctx.GetDistCalc().Distance(
-                ctx.MakePoint(bbox.GetMinX(), bbox.GetMinY()), bbox.GetMaxX(), bbox.GetMaxY());
-            return diagonalDist * 0.5 * distErrPct;
-        }
-
-        /// <summary>
-        /// Gets the error distance that specifies how precise the query shape is. This
-        /// looks at {@link #getDistErr()}, {@link #getDistErrPct()}, and {@code
-        /// defaultDistErrPct}.
-        /// </summary>
-        /// <param name="ctx"></param>
-        /// <param name="defaultDistErrPct">0 to 0.5</param>
-        /// <returns>>= 0</returns>
-        public double ResolveDistErr(SpatialContext ctx, double defaultDistErrPct)
-        {
-            if (DistErr != null)
-                return DistErr.Value;
-            double? distErrPct = (this.distErrPct ?? defaultDistErrPct);
-            return CalcDistanceFromErrPct(Shape, distErrPct.Value, ctx);
-        }
-
-        /// <summary>
-        /// Check if the arguments make sense -- throw an exception if not
-        /// </summary>
-        public void Validate()
-        {
-            if (Operation.IsTargetNeedsArea() && !Shape.HasArea())
-            {
-                throw new ArgumentException(Operation + " only supports geometry with area");
-            }
-        }
-
-        public override String ToString()
-        {
-            return SpatialArgsParser.WriteSpatialArgs(this);
-        }
-
-        //------------------------------------------------
-        // Getters & Setters
-        //------------------------------------------------
-
-        public Shape Shape { get; set; }
-
-        /// <summary>
-        /// A measure of acceptable error of the shape as a fraction. This effectively
-        /// inflates the size of the shape but should not shrink it.
-        /// <p/>
-        /// The default is {@link #DEFAULT_DIST_PRECISION}
-        /// </summary>
-        /// <returns>0 to 0.5</returns>
-        public double? DistErrPct
-        {
-            get { return distErrPct; }
-            set
-            {
-                if (value != null)
-                    distErrPct = value.Value;
-            }
-        }
-        private double? distErrPct;
-
-        /// <summary>
-        /// The acceptable error of the shape.  This effectively inflates the
-        /// size of the shape but should not shrink it.
-        /// </summary>
-        /// <returns>>= 0</returns>
-        public double? DistErr { get; set; }
-    }
-}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/Queries/SpatialArgsParser.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Queries/SpatialArgsParser.cs b/src/Lucene.Net.Spatial/Queries/SpatialArgsParser.cs
deleted file mode 100644
index b6faa9f..0000000
--- a/src/Lucene.Net.Spatial/Queries/SpatialArgsParser.cs
+++ /dev/null
@@ -1,140 +0,0 @@
-\ufeff/*
- * 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 System.Collections.Generic;
-using System.Text;
-using Spatial4n.Core.Context;
-using Spatial4n.Core.Io;
-using Spatial4n.Core.Shapes;
-
-namespace Lucene.Net.Spatial.Queries
-{
-    public class SpatialArgsParser
-    {
-        public const String DIST_ERR_PCT = "distErrPct";
-        public const String DIST_ERR = "distErr";
-
-        /// <summary>
-        /// Writes a close approximation to the parsed input format.
-        /// </summary>
-        /// <param name="args"></param>
-        /// <returns></returns>
-        public static String WriteSpatialArgs(SpatialArgs args)
-        {
-            var str = new StringBuilder();
-            str.Append(args.Operation.Name);
-            str.Append('(');
-            str.Append(args.Shape);
-            if (args.DistErrPct != null)
-                str.Append(" distErrPct=").Append(String.Format("{0:0.00}%", args.DistErrPct * 100d));
-            if (args.DistErr != null)
-                str.Append(" distErr=").Append(args.DistErr);
-            str.Append(')');
-            return str.ToString();
-        }
-
-        /// <summary>
-        /// Parses a string such as "Intersects(-10,20,-8,22) distErrPct=0.025".
-        /// </summary>
-        /// <param name="v"></param>
-        /// <param name="ctx"></param>
-        /// <returns></returns>
-        public SpatialArgs Parse(String v, SpatialContext ctx)
-        {
-            int idx = v.IndexOf('(');
-            int edx = v.LastIndexOf(')');
-
-            if (idx < 0 || idx > edx)
-            {
-                throw new ArgumentException("missing parens: " + v);
-            }
-
-            SpatialOperation op = SpatialOperation.Get(v.Substring(0, idx).Trim());
-
-            //Substring in .NET is (startPosn, length), But in Java it's (startPosn, endPosn)
-            //see http://docs.oracle.com/javase/1.4.2/docs/api/java/lang/String.html#substring(int, int)
-            String body = v.Substring(idx + 1, edx - (idx + 1)).Trim();
-            if (body.Length < 1)
-            {
-                throw new ArgumentException("missing body : " + v);
-            }
-
-            var shape = ctx.ReadShape(body);
-            var args = new SpatialArgs(op, shape);
-
-            if (v.Length > (edx + 1))
-            {
-                body = v.Substring(edx + 1).Trim();
-                if (body.Length > 0)
-                {
-                    Dictionary<String, String> aa = ParseMap(body);
-                    args.DistErrPct = ReadDouble(aa["distErrPct"]); aa.Remove(DIST_ERR_PCT);
-                    args.DistErr = ReadDouble(aa["distErr"]); aa.Remove(DIST_ERR);
-                    if (aa.Count != 0)
-                    {
-                        throw new ArgumentException("unused parameters: " + aa);
-                    }
-                }
-            }
-            args.Validate();
-            return args;
-        }
-
-        protected static double? ReadDouble(String v)
-        {
-            double val;
-            return double.TryParse(v, out val) ? val : (double?)null;
-        }
-
-        protected static bool ReadBool(String v, bool defaultValue)
-        {
-            bool ret;
-            return bool.TryParse(v, out ret) ? ret : defaultValue;
-        }
-
-        /// <summary>
-        /// Parses "a=b c=d f" (whitespace separated) into name-value pairs. If there
-        /// is no '=' as in 'f' above then it's short for f=f.
-        /// </summary>
-        /// <param name="body"></param>
-        /// <returns></returns>
-        protected static Dictionary<String, String> ParseMap(String body)
-        {
-            var map = new Dictionary<String, String>();
-            int tokenPos = 0;
-            var st = body.Split(new[] { ' ', '\n', '\t' }, StringSplitOptions.RemoveEmptyEntries);
-            while (tokenPos < st.Length)
-            {
-                String a = st[tokenPos++];
-                int idx = a.IndexOf('=');
-                if (idx > 0)
-                {
-                    String k = a.Substring(0, idx);
-                    String v = a.Substring(idx + 1);
-                    map[k] = v;
-                }
-                else
-                {
-                    map[a] = a;
-                }
-            }
-            return map;
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/Queries/SpatialOperation.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Queries/SpatialOperation.cs b/src/Lucene.Net.Spatial/Queries/SpatialOperation.cs
deleted file mode 100644
index 26f57d3..0000000
--- a/src/Lucene.Net.Spatial/Queries/SpatialOperation.cs
+++ /dev/null
@@ -1,116 +0,0 @@
-\ufeff/* See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * Esri Inc. 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 System.Collections.Generic;
-using System.Globalization;
-using System.Linq;
-using Spatial4n.Core.Exceptions;
-
-namespace Lucene.Net.Spatial.Queries
-{
-    public class SpatialOperation
-    {
-        // Private registry
-        private static readonly Dictionary<String, SpatialOperation> registry = new Dictionary<string, SpatialOperation>();
-        private static readonly IList<SpatialOperation> list = new List<SpatialOperation>();
-
-        // Geometry Operations
-
-        /// <summary>
-        /// Bounding box of the *indexed* shape.
-        /// </summary>
-        public static readonly SpatialOperation BBoxIntersects = new SpatialOperation("BBoxIntersects", true, false, false);
-
-        /// <summary>
-        /// Bounding box of the *indexed* shape.
-        /// </summary>
-        public static readonly SpatialOperation BBoxWithin = new SpatialOperation("BBoxWithin", true, false, false);
-
-        public static readonly SpatialOperation Contains = new SpatialOperation("Contains", true, true, false);
-        public static readonly SpatialOperation Intersects = new SpatialOperation("Intersects", true, false, false);
-        public static readonly SpatialOperation IsEqualTo = new SpatialOperation("IsEqualTo", false, false, false);
-        public static readonly SpatialOperation IsDisjointTo = new SpatialOperation("IsDisjointTo", false, false, false);
-        public static readonly SpatialOperation IsWithin = new SpatialOperation("IsWithin", true, false, true);
-        public static readonly SpatialOperation Overlaps = new SpatialOperation("Overlaps", true, false, true);
-
-        // Member variables
-        private readonly bool scoreIsMeaningful;
-        private readonly bool sourceNeedsArea;
-        private readonly bool targetNeedsArea;
-        private readonly String name;
-
-        protected SpatialOperation(String name, bool scoreIsMeaningful, bool sourceNeedsArea, bool targetNeedsArea)
-        {
-            this.name = name;
-            this.scoreIsMeaningful = scoreIsMeaningful;
-            this.sourceNeedsArea = sourceNeedsArea;
-            this.targetNeedsArea = targetNeedsArea;
-            registry[name] = this;
-            registry[name.ToUpper(CultureInfo.CreateSpecificCulture("en-US"))] = this;
-            list.Add(this);
-        }
-
-        public static SpatialOperation Get(String v)
-        {
-            SpatialOperation op;
-            if (!registry.TryGetValue(v, out op) || op == null)
-            {
-                if (!registry.TryGetValue(v.ToUpper(CultureInfo.CreateSpecificCulture("en-US")), out op) || op == null)
-                    throw new ArgumentException("Unknown Operation: " + v, "v");
-            }
-            return op;
-        }
-
-        public static IList<SpatialOperation> Values()
-        {
-            return list;
-        }
-
-        public static bool Is(SpatialOperation op, params SpatialOperation[] tst)
-        {
-            return tst.Any(t => op == t);
-        }
-
-
-        // ================================================= Getters / Setters =============================================
-
-        public bool IsScoreIsMeaningful()
-        {
-            return scoreIsMeaningful;
-        }
-
-        public bool IsSourceNeedsArea()
-        {
-            return sourceNeedsArea;
-        }
-
-        public bool IsTargetNeedsArea()
-        {
-            return targetNeedsArea;
-        }
-
-        public String Name
-        {
-            get { return name; }
-        }
-
-        public override String ToString()
-        {
-            return name;
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/Queries/UnsupportedSpatialOperation.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Queries/UnsupportedSpatialOperation.cs b/src/Lucene.Net.Spatial/Queries/UnsupportedSpatialOperation.cs
deleted file mode 100644
index 35949bc..0000000
--- a/src/Lucene.Net.Spatial/Queries/UnsupportedSpatialOperation.cs
+++ /dev/null
@@ -1,30 +0,0 @@
-\ufeff/*
- * 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;
-
-namespace Lucene.Net.Spatial.Queries
-{
-    [Serializable]
-    public class UnsupportedSpatialOperation : InvalidOperationException
-    {
-        public UnsupportedSpatialOperation(SpatialOperation op)
-            : base(op.Name)
-        {
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialArgs.cs b/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
new file mode 100644
index 0000000..11a3114
--- /dev/null
+++ b/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
@@ -0,0 +1,136 @@
+\ufeff/*
+ * 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 Spatial4n.Core.Context;
+using Spatial4n.Core.Shapes;
+
+namespace Lucene.Net.Spatial.Query
+{
+    public class SpatialArgs
+    {
+        public static readonly double DEFAULT_DISTERRPCT = 0.025d;
+
+        public SpatialOperation Operation { get; set; }
+
+        public SpatialArgs(SpatialOperation operation, Shape shape)
+        {
+            if (operation == null || shape == null)
+                throw new ArgumentException("operation and shape are required");
+            this.Operation = operation;
+            this.Shape = shape;
+        }
+
+        /// <summary>
+        /// Computes the distance given a shape and the {@code distErrPct}.  The
+        /// algorithm is the fraction of the distance from the center of the query
+        /// shape to its furthest bounding box corner.
+        /// </summary>
+        /// <param name="shape">Mandatory.</param>
+        /// <param name="distErrPct">0 to 0.5</param>
+        /// <param name="ctx">Mandatory</param>
+        /// <returns>A distance (in degrees).</returns>
+        public static double CalcDistanceFromErrPct(Shape shape, double distErrPct, SpatialContext ctx)
+        {
+            if (distErrPct < 0 || distErrPct > 0.5)
+            {
+                throw new ArgumentException("distErrPct " + distErrPct + " must be between [0 to 0.5]", "distErrPct");
+            }
+            if (distErrPct == 0 || shape is Point)
+            {
+                return 0;
+            }
+            Rectangle bbox = shape.GetBoundingBox();
+
+            //Compute the distance from the center to a corner.  Because the distance
+            // to a bottom corner vs a top corner can vary in a geospatial scenario,
+            // take the closest one (greater precision).
+            Point ctr = bbox.GetCenter();
+            double y = (ctr.GetY() >= 0 ? bbox.GetMaxY() : bbox.GetMinY());
+            double diagonalDist = ctx.GetDistCalc().Distance(ctr, bbox.GetMaxX(), y);
+            return diagonalDist * distErrPct;
+        }
+
+        /// <summary>
+        /// Gets the error distance that specifies how precise the query shape is. This
+        /// looks at {@link #getDistErr()}, {@link #getDistErrPct()}, and {@code
+        /// defaultDistErrPct}.
+        /// </summary>
+        /// <param name="ctx"></param>
+        /// <param name="defaultDistErrPct">0 to 0.5</param>
+        /// <returns>>= 0</returns>
+        public double ResolveDistErr(SpatialContext ctx, double defaultDistErrPct)
+        {
+            if (DistErr != null)
+                return DistErr.Value;
+            double distErrPct = (this.distErrPct ?? defaultDistErrPct);
+            return CalcDistanceFromErrPct(Shape, distErrPct, ctx);
+        }
+
+        /// <summary>
+        /// Check if the arguments make sense -- throw an exception if not
+        /// </summary>
+        public void Validate()
+        {
+            if (Operation.IsTargetNeedsArea() && !Shape.HasArea())
+            {
+                throw new ArgumentException(Operation + " only supports geometry with area");
+            }
+
+            if (DistErr != null && DistErrPct != null)
+            {
+                throw new ArgumentException("Only DistErr or DistErrPct can be specified.");
+            }
+        }
+
+        public override String ToString()
+        {
+            return SpatialArgsParser.WriteSpatialArgs(this);
+        }
+
+        //------------------------------------------------
+        // Getters & Setters
+        //------------------------------------------------
+
+        public Shape Shape { get; set; }
+
+        /// <summary>
+        /// A measure of acceptable error of the shape as a fraction. This effectively
+        /// inflates the size of the shape but should not shrink it.
+        /// <p/>
+        /// The default is {@link #DEFAULT_DIST_PRECISION}
+        /// </summary>
+        /// <returns>0 to 0.5</returns>
+        public double? DistErrPct
+        {
+            get { return distErrPct; }
+            set
+            {
+                if (value != null)
+                    distErrPct = value.Value;
+            }
+        }
+        private double? distErrPct;
+
+        /// <summary>
+        /// The acceptable error of the shape.  This effectively inflates the
+        /// size of the shape but should not shrink it.
+        /// </summary>
+        /// <returns>>= 0</returns>
+        public double? DistErr { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs b/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
new file mode 100644
index 0000000..6f0b433
--- /dev/null
+++ b/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
@@ -0,0 +1,144 @@
+\ufeff/*
+ * 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 System.Collections.Generic;
+using System.Text;
+using Lucene.Net.Support;
+using Spatial4n.Core.Context;
+
+namespace Lucene.Net.Spatial.Query
+{
+    public class SpatialArgsParser
+    {
+        public const String DIST_ERR_PCT = "distErrPct";
+        public const String DIST_ERR = "distErr";
+
+        /// <summary>
+        /// Writes a close approximation to the parsed input format.
+        /// </summary>
+        /// <param name="args"></param>
+        /// <returns></returns>
+        public static String WriteSpatialArgs(SpatialArgs args)
+        {
+            var str = new StringBuilder();
+            str.Append(args.Operation.Name);
+            str.Append('(');
+            str.Append(args.Shape);
+            if (args.DistErrPct != null)
+                str.Append(" distErrPct=").Append(String.Format("{0:0.00}%", args.DistErrPct * 100d));
+            if (args.DistErr != null)
+                str.Append(" distErr=").Append(args.DistErr);
+            str.Append(')');
+            return str.ToString();
+        }
+
+        /// <summary>
+        /// Parses a string such as "Intersects(-10,20,-8,22) distErrPct=0.025".
+        /// </summary>
+        /// <param name="v"></param>
+        /// <param name="ctx"></param>
+        /// <returns></returns>
+        public SpatialArgs Parse(String v, SpatialContext ctx)
+        {
+            int idx = v.IndexOf('(');
+            int edx = v.LastIndexOf(')');
+
+            if (idx < 0 || idx > edx)
+            {
+                throw new ArgumentException("missing parens: " + v);
+            }
+
+            SpatialOperation op = SpatialOperation.Get(v.Substring(0, idx).Trim());
+
+            //Substring in .NET is (startPosn, length), But in Java it's (startPosn, endPosn)
+            //see http://docs.oracle.com/javase/1.4.2/docs/api/java/lang/String.html#substring(int, int)
+            String body = v.Substring(idx + 1, edx - (idx + 1)).Trim();
+            if (body.Length < 1)
+            {
+                throw new ArgumentException("missing body : " + v);
+            }
+
+            var shape = ctx.ReadShape(body);
+            var args = new SpatialArgs(op, shape);
+
+            if (v.Length > (edx + 1))
+            {
+                body = v.Substring(edx + 1).Trim();
+                if (body.Length > 0)
+                {
+                    Dictionary<String, String> aa = ParseMap(body);
+                    args.DistErrPct = ReadDouble(aa[DIST_ERR_PCT]);
+                    aa.Remove(DIST_ERR_PCT);
+
+                    args.DistErr = ReadDouble(aa[DIST_ERR]);
+                    aa.Remove(DIST_ERR);
+
+                    if (aa.Count != 0)
+                    {
+                        throw new ArgumentException("unused parameters: " + aa);
+                    }
+                }
+            }
+            args.Validate();
+            return args;
+        }
+
+        protected static double? ReadDouble(String v)
+        {
+            double val;
+            return double.TryParse(v, out val) ? val : (double?)null;
+        }
+
+        protected static bool ReadBool(String v, bool defaultValue)
+        {
+            bool ret;
+            return bool.TryParse(v, out ret) ? ret : defaultValue;
+        }
+
+        /// <summary>
+        /// Parses "a=b c=d f" (whitespace separated) into name-value pairs. If there
+        /// is no '=' as in 'f' above then it's short for f=f.
+        /// </summary>
+        /// <param name="body"></param>
+        /// <returns></returns>
+        protected static Dictionary<String, String> ParseMap(String body)
+        {
+            var map = new Dictionary<String, String>();
+            StringTokenizer st = new StringTokenizer(body, " \n\t");
+
+            while (st.HasMoreTokens())
+            {
+                String a = st.NextToken();
+                int idx = a.IndexOf('=');
+                if (idx > 0)
+                {
+                    String k = a.Substring(0, idx);
+                    String v = a.Substring(idx + 1);
+                    map[k] = v;
+                }
+                else
+                {
+                    map[a] = a;
+                }
+            }
+
+            return map;
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialOperation.cs b/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
new file mode 100644
index 0000000..16c28a4
--- /dev/null
+++ b/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
@@ -0,0 +1,114 @@
+\ufeff/* See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * Esri Inc. 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 System.Collections.Generic;
+using System.Globalization;
+using System.Linq;
+
+namespace Lucene.Net.Spatial.Query
+{
+    public class SpatialOperation
+    {
+        // Private registry
+        private static readonly Dictionary<String, SpatialOperation> registry = new Dictionary<string, SpatialOperation>();
+        private static readonly IList<SpatialOperation> list = new List<SpatialOperation>();
+
+        // Geometry Operations
+
+        /// <summary>
+        /// Bounding box of the *indexed* shape.
+        /// </summary>
+        public static readonly SpatialOperation BBoxIntersects = new SpatialOperation("BBoxIntersects", true, false, false);
+
+        /// <summary>
+        /// Bounding box of the *indexed* shape.
+        /// </summary>
+        public static readonly SpatialOperation BBoxWithin = new SpatialOperation("BBoxWithin", true, false, false);
+
+        public static readonly SpatialOperation Contains = new SpatialOperation("Contains", true, true, false);
+        public static readonly SpatialOperation Intersects = new SpatialOperation("Intersects", true, false, false);
+        public static readonly SpatialOperation IsEqualTo = new SpatialOperation("IsEqualTo", false, false, false);
+        public static readonly SpatialOperation IsDisjointTo = new SpatialOperation("IsDisjointTo", false, false, false);
+        public static readonly SpatialOperation IsWithin = new SpatialOperation("IsWithin", true, false, true);
+        public static readonly SpatialOperation Overlaps = new SpatialOperation("Overlaps", true, false, true);
+
+        // Member variables
+        private readonly bool scoreIsMeaningful;
+        private readonly bool sourceNeedsArea;
+        private readonly bool targetNeedsArea;
+        private readonly String name;
+
+        protected SpatialOperation(String name, bool scoreIsMeaningful, bool sourceNeedsArea, bool targetNeedsArea)
+        {
+            this.name = name;
+            this.scoreIsMeaningful = scoreIsMeaningful;
+            this.sourceNeedsArea = sourceNeedsArea;
+            this.targetNeedsArea = targetNeedsArea;
+            registry[name] = this;
+            registry[name.ToUpper(CultureInfo.CreateSpecificCulture("en-US"))] = this;
+            list.Add(this);
+        }
+
+        public static SpatialOperation Get(String v)
+        {
+            SpatialOperation op;
+            if (!registry.TryGetValue(v, out op) || op == null)
+            {
+                if (!registry.TryGetValue(v.ToUpper(CultureInfo.CreateSpecificCulture("en-US")), out op) || op == null)
+                    throw new ArgumentException("Unknown Operation: " + v, "v");
+            }
+            return op;
+        }
+
+        public static IList<SpatialOperation> Values()
+        {
+            return list;
+        }
+
+        public static bool Is(SpatialOperation op, params SpatialOperation[] tst)
+        {
+            return tst.Any(t => op == t);
+        }
+
+        // ================================================= Getters / Setters =============================================
+
+        public bool IsScoreIsMeaningful()
+        {
+            return scoreIsMeaningful;
+        }
+
+        public bool IsSourceNeedsArea()
+        {
+            return sourceNeedsArea;
+        }
+
+        public bool IsTargetNeedsArea()
+        {
+            return targetNeedsArea;
+        }
+
+        public String Name
+        {
+            get { return name; }
+        }
+
+        public override String ToString()
+        {
+            return name;
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs b/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs
new file mode 100644
index 0000000..92ae852
--- /dev/null
+++ b/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs
@@ -0,0 +1,30 @@
+\ufeff/*
+ * 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;
+
+namespace Lucene.Net.Spatial.Query
+{
+    [Serializable]
+    public class UnsupportedSpatialOperation : InvalidOperationException
+    {
+        public UnsupportedSpatialOperation(SpatialOperation op)
+            : base(op.Name)
+        {
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
new file mode 100644
index 0000000..d79c02c
--- /dev/null
+++ b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
@@ -0,0 +1,69 @@
+\ufeff/*
+ * 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 System.Collections.Generic;
+using System.IO;
+using System.Linq;
+using System.Text;
+using System.Threading.Tasks;
+using Lucene.Net.Documents;
+using Lucene.Net.Queries.Function;
+using Lucene.Net.Search;
+using Lucene.Net.Spatial;
+using Lucene.Net.Spatial.Query;
+using Lucene.Net.Support;
+using Lucene.Net.Util;
+using Spatial4n.Core.Context;
+using Spatial4n.Core.Shapes;
+
+namespace Lucene.Net.Spatial.Serialized
+{
+    /// <summary>
+    /// A SpatialStrategy based on serializing a Shape stored into BinaryDocValues.
+    /// This is not at all fast; it's designed to be used in conjuction with another index based
+    /// SpatialStrategy that is approximated(like { @link org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy})
+    /// to add precision or eventually make more specific / advanced calculations on the per-document
+    /// geometry.
+    /// The serialization uses Spatial4j's {@link com.spatial4j.core.io.BinaryCodec}.
+    ///
+    /// @lucene.experimental
+    /// </summary>
+    public class SerializedDVStrategy : SpatialStrategy
+    {
+        public SerializedDVStrategy(SpatialContext ctx, string fieldName) : base(ctx, fieldName)
+        {
+            throw new NotImplementedException();
+        }
+
+        public override Field[] CreateIndexableFields(Shape shape)
+        {
+            throw new NotImplementedException();
+        }
+
+        public override ValueSource MakeDistanceValueSource(Point queryPoint, double multiplier)
+        {
+            throw new NotImplementedException();
+        }
+
+        public override Filter MakeFilter(SpatialArgs args)
+        {
+            throw new NotImplementedException();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/SpatialStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/SpatialStrategy.cs b/src/Lucene.Net.Spatial/SpatialStrategy.cs
index 2fb8689..76f827f 100644
--- a/src/Lucene.Net.Spatial/SpatialStrategy.cs
+++ b/src/Lucene.Net.Spatial/SpatialStrategy.cs
@@ -20,7 +20,7 @@ using Lucene.Net.Documents;
 using Lucene.Net.Queries.Function;
 using Lucene.Net.Queries.Function.ValueSources;
 using Lucene.Net.Search;
-using Lucene.Net.Spatial.Queries;
+using Lucene.Net.Spatial.Query;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Shapes;
 

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/9e566a89/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
index aec23a0..213e95c 100644
--- a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
+++ b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
@@ -19,7 +19,7 @@ using System;
 using Lucene.Net.Documents;
 using Lucene.Net.Queries.Function;
 using Lucene.Net.Search;
-using Lucene.Net.Spatial.Queries;
+using Lucene.Net.Spatial.Query;
 using Lucene.Net.Spatial.Util;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Shapes;


[04/26] lucenenet git commit: Fixing namespace issues

Posted by ni...@apache.org.
Fixing namespace issues


Project: http://git-wip-us.apache.org/repos/asf/lucenenet/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucenenet/commit/6c7cc799
Tree: http://git-wip-us.apache.org/repos/asf/lucenenet/tree/6c7cc799
Diff: http://git-wip-us.apache.org/repos/asf/lucenenet/diff/6c7cc799

Branch: refs/heads/master
Commit: 6c7cc79984c285d161c8aef44e00f86b445995c1
Parents: 9e566a8
Author: Naz Junaid <na...@gmail.com>
Authored: Sat Aug 13 23:23:12 2016 +0100
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Tue Nov 15 19:17:54 2016 +0700

----------------------------------------------------------------------
 src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/6c7cc799/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
index 213e95c..b4b2e87 100644
--- a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
+++ b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
@@ -130,7 +130,7 @@ namespace Lucene.Net.Spatial.Vector
         }
 
         //TODO this is basically old code that hasn't been verified well and should probably be removed
-        public Query MakeQueryDistanceScore(SpatialArgs args)
+        public Search.Query MakeQueryDistanceScore(SpatialArgs args)
         {
             // For starters, just limit the bbox
             var shape = args.Shape;
@@ -146,7 +146,7 @@ namespace Lucene.Net.Spatial.Vector
 
             ValueSource valueSource = null;
 
-            Query spatial = null;
+            Search.Query spatial = null;
             SpatialOperation op = args.Operation;
 
             if (SpatialOperation.Is(op,
@@ -190,7 +190,7 @@ namespace Lucene.Net.Spatial.Vector
             {
                 valueSource = MakeDistanceValueSource(shape.GetCenter());
             }
-            Query spatialRankingQuery = new FunctionQuery(valueSource);
+            Search.Query spatialRankingQuery = new FunctionQuery(valueSource);
             var bq = new BooleanQuery();
             bq.Add(spatial, BooleanClause.Occur.MUST);
             bq.Add(spatialRankingQuery, BooleanClause.Occur.MUST);
@@ -213,7 +213,7 @@ namespace Lucene.Net.Spatial.Vector
         /// Constructs a query to retrieve documents that fully contain the input envelope.
         /// </summary>
         /// <param name="bbox"></param>
-        private Query MakeWithin(Rectangle bbox)
+        private Search.Query MakeWithin(Rectangle bbox)
         {
             var bq = new BooleanQuery();
             const BooleanClause.Occur MUST = BooleanClause.Occur.MUST;
@@ -247,12 +247,12 @@ namespace Lucene.Net.Spatial.Vector
         /// Constructs a query to retrieve documents that fully contain the input envelope.
         /// </summary>
         /// <param name="bbox"></param>
-        private Query MakeDisjoint(Rectangle bbox)
+        private Search.Query MakeDisjoint(Rectangle bbox)
         {
             if (bbox.GetCrossesDateLine())
                 throw new InvalidOperationException("MakeDisjoint doesn't handle dateline cross");
-            Query qX = RangeQuery(fieldNameX, bbox.GetMinX(), bbox.GetMaxX());
-            Query qY = RangeQuery(fieldNameY, bbox.GetMinY(), bbox.GetMaxY());
+            Search.Query qX = RangeQuery(fieldNameX, bbox.GetMinX(), bbox.GetMaxX());
+            Search.Query qY = RangeQuery(fieldNameY, bbox.GetMinY(), bbox.GetMaxY());
             var bq = new BooleanQuery { { qX, BooleanClause.Occur.MUST_NOT }, { qY, BooleanClause.Occur.MUST_NOT } };
             return bq;
         }


[07/26] lucenenet git commit: Setup InternalsVisibleTo Lucene.Net.Spatial <- Lucene.Net.Tests.Spatial

Posted by ni...@apache.org.
Setup InternalsVisibleTo Lucene.Net.Spatial <- Lucene.Net.Tests.Spatial


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

Branch: refs/heads/master
Commit: bd9e92d9e8ccc014544f5e68235c41d9066aba4c
Parents: 415cd5c
Author: Shad Storhaug <sh...@shadstorhaug.com>
Authored: Wed Nov 16 06:06:43 2016 +0700
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Wed Nov 16 06:06:43 2016 +0700

----------------------------------------------------------------------
 src/Lucene.Net.Spatial/Properties/AssemblyInfo.cs | 18 ++++++++++--------
 1 file changed, 10 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/bd9e92d9/src/Lucene.Net.Spatial/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Properties/AssemblyInfo.cs b/src/Lucene.Net.Spatial/Properties/AssemblyInfo.cs
index a5b4d0a..8c8f6d3 100644
--- a/src/Lucene.Net.Spatial/Properties/AssemblyInfo.cs
+++ b/src/Lucene.Net.Spatial/Properties/AssemblyInfo.cs
@@ -16,6 +16,7 @@
  */
 
 using System.Reflection;
+using System.Runtime.CompilerServices;
 using System.Runtime.InteropServices;
 using System.Security;
 
@@ -40,6 +41,9 @@ using System.Security;
 // The following GUID is for the ID of the typelib if this project is exposed to COM
 [assembly: Guid("5c8e810f-4bf7-472f-9785-8d80a0de6ea8")]
 
+// for testing
+[assembly: InternalsVisibleTo("Lucene.Net.Tests.Spatial")]
+
 // Version information for an assembly consists of the following four values:
 //
 //      Major Version
@@ -50,14 +54,12 @@ using System.Security;
 // You can specify all the values or you can default the Build and Revision Numbers 
 // by using the '*' as shown below:
 // [assembly: AssemblyVersion("1.0.*")]
-[assembly: AssemblyInformationalVersionAttribute("3.0.3")]
-
-[assembly: AssemblyVersion("3.0.3")]
-[assembly: AssemblyFileVersion("3.0.3")]
+[assembly: AssemblyVersion("4.8.0")]
+[assembly: AssemblyFileVersion("4.8.0")]
 
 
-[assembly: AssemblyDelaySign(false)]
-[assembly: AssemblyKeyFile("")]
-[assembly: AssemblyKeyName("")]
+//[assembly: AssemblyDelaySign(false)]
+//[assembly: AssemblyKeyFile("")]
+//[assembly: AssemblyKeyName("")]
 
-[assembly: AllowPartiallyTrustedCallers]
+//[assembly: AllowPartiallyTrustedCallers]


[08/26] lucenenet git commit: Added missing Spatial classes, ported Spatial tests, and fixed several bugs

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Test-Files/cities-Intersects-BBox.txt
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Test-Files/cities-Intersects-BBox.txt b/src/Lucene.Net.Tests.Spatial/Test-Files/cities-Intersects-BBox.txt
new file mode 100644
index 0000000..e85748c
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Test-Files/cities-Intersects-BBox.txt
@@ -0,0 +1,3 @@
+[San Francisco] G5391959 @ Intersects(ENVELOPE(-122.524918, -122.360123, 37.817108, 37.674973))
+[Wellington] G2179537 @ Intersects(ENVELOPE(174.711456, 174.854279, -41.213837, -41.360779))
+[Barcelona] G6544100 G3128760  @  Intersects(ENVELOPE(2.127228, 2.226105, 41.408844, 41.333313))
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Test-Files/simple-Queries-BBox.txt
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Test-Files/simple-Queries-BBox.txt b/src/Lucene.Net.Tests.Spatial/Test-Files/simple-Queries-BBox.txt
new file mode 100644
index 0000000..7aacc9b
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Test-Files/simple-Queries-BBox.txt
@@ -0,0 +1,8 @@
+C5 @ IsWithin(ENVELOPE(-6, 6, 6, -6))
+C5 @ BBoxWithin(ENVELOPE(-6, 6, 6, -6))
+C10 @ Contains(ENVELOPE(-6, 6, 6, -6))
+C10 @ IsEqualTo(ENVELOPE(-10, 10, 10, -10))
+C5 C10 @ Intersects(ENVELOPE(-2, 2, 2, -2))
+C5 C10 @ Overlaps(ENVELOPE(-2, 2, 2, -2))
+C5 C10 @ BBoxIntersects(ENVELOPE(-2, 2, 2, -2))
+NW15 @ IsDisjointTo(ENVELOPE(-10, 10, 10, -10))
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Test-Files/states-Intersects-BBox.txt
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Test-Files/states-Intersects-BBox.txt b/src/Lucene.Net.Tests.Spatial/Test-Files/states-Intersects-BBox.txt
new file mode 100644
index 0000000..a45df7b
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Test-Files/states-Intersects-BBox.txt
@@ -0,0 +1,3 @@
+WY CO	@ Intersects(ENVELOPE(-106.964844, -105.734375, 42.800781, 39.460938))
+TX @ Intersects(ENVELOPE(-99.669922, -98.439453, 32.253906, 30.583984))
+MS TX LA @ Intersects(ENVELOPE(-95.363281, -90.133789, 32.473633, 29.792969))
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Test-Files/states-IsWithin-BBox.txt
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Test-Files/states-IsWithin-BBox.txt b/src/Lucene.Net.Tests.Spatial/Test-Files/states-IsWithin-BBox.txt
new file mode 100644
index 0000000..6a504da
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Test-Files/states-IsWithin-BBox.txt
@@ -0,0 +1,4 @@
+KS	@ IsWithin(ENVELOPE(-103.493164, -93.825195, 41.086914, 36.208984))
+WA @ IsWithin(ENVELOPE(-126.916016, -115.314453, 50.688965, 44.36084))
+MA CT RI @ IsWithin(ENVELOPE(-73.894043, -69.521484, 43.198242, 40.825195))
+AL GA @ IsWithin(ENVELOPE(-89.472656, -80.244141, 35.90332, 29.311523))
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/TestTestFramework.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/TestTestFramework.cs b/src/Lucene.Net.Tests.Spatial/TestTestFramework.cs
new file mode 100644
index 0000000..72f99c1
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/TestTestFramework.cs
@@ -0,0 +1,63 @@
+\ufeffusing Lucene.Net.Spatial.Queries;
+using Lucene.Net.Util;
+using NUnit.Framework;
+using Spatial4n.Core.Context;
+using Spatial4n.Core.Shapes;
+using System;
+using System.Collections.Generic;
+using System.IO;
+
+namespace Lucene.Net.Spatial
+{
+    /*
+     * 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.
+     */
+
+    public class TestTestFramework : LuceneTestCase
+    {
+        [Test]
+        public virtual void TestQueries()
+        {
+            String name = StrategyTestCase.RESOURCE_PATH + StrategyTestCase.QTEST_Cities_Intersects_BBox;
+
+            Stream @in = GetType().Assembly.GetManifestResourceStream(name);
+            SpatialContext ctx = SpatialContext.GEO;
+            IEnumerator<SpatialTestQuery> iter = SpatialTestQuery.GetTestQueries(
+                new SpatialArgsParser(), ctx, name, @in);//closes the InputStream
+            List<SpatialTestQuery> tests = new List<SpatialTestQuery>();
+            while (iter.MoveNext())
+            {
+                tests.Add(iter.Current);
+            }
+            assertEquals(3, tests.size());
+
+            SpatialTestQuery sf = tests[0];
+            // assert
+            assertEquals(1, sf.ids.size());
+            assertTrue(sf.ids[0].equals("G5391959"));
+            assertTrue(sf.args.Shape is IRectangle);
+            assertEquals(SpatialOperation.Intersects, sf.args.Operation);
+        }
+
+        [Test]
+        public virtual void SpatialExample_Mem()
+        {
+            //kind of a hack so that SpatialExample is tested despite
+            // it not starting or ending with "Test".
+            SpatialExample.Main(null);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Vector/TestPointVectorStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Vector/TestPointVectorStrategy.cs b/src/Lucene.Net.Tests.Spatial/Vector/TestPointVectorStrategy.cs
new file mode 100644
index 0000000..a43faf0
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Vector/TestPointVectorStrategy.cs
@@ -0,0 +1,61 @@
+\ufeffusing Lucene.Net.Search;
+using Lucene.Net.Spatial.Queries;
+using NUnit.Framework;
+using Spatial4n.Core.Context;
+using Spatial4n.Core.Shapes;
+using System;
+
+namespace Lucene.Net.Spatial.Vector
+{
+    /*
+     * 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.
+     */
+
+    public class TestPointVectorStrategy : StrategyTestCase
+    {
+        public override void SetUp()
+        {
+            base.SetUp();
+            this.ctx = SpatialContext.GEO;
+            this.strategy = new PointVectorStrategy(ctx, GetType().Name);
+        }
+
+        [Test]
+        public virtual void TestCircleShapeSupport()
+        {
+            ICircle circle = ctx.MakeCircle(ctx.MakePoint(0, 0), 10);
+            SpatialArgs args = new SpatialArgs(SpatialOperation.Intersects, circle);
+            Query query = this.strategy.MakeQuery(args);
+
+            assertNotNull(query);
+        }
+
+        [Test]
+        public virtual void TestInvalidQueryShape()
+        {
+            IPoint point = ctx.MakePoint(0, 0);
+            SpatialArgs args = new SpatialArgs(SpatialOperation.Intersects, point);
+            Assert.Throws<NotSupportedException>(() => this.strategy.MakeQuery(args));
+        }
+
+        [Test]
+        public virtual void TestCitiesIntersectsBBox()
+        {
+            getAddAndVerifyIndexedDocuments(DATA_WORLD_CITIES_POINTS);
+            executeQueries(SpatialMatchConcern.FILTER, QTEST_Cities_Intersects_BBox);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/packages.config
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/packages.config b/src/Lucene.Net.Tests.Spatial/packages.config
new file mode 100644
index 0000000..139d513
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/packages.config
@@ -0,0 +1,4 @@
+\ufeff<?xml version="1.0" encoding="utf-8"?>
+<packages>
+  <package id="NUnit" version="2.6.3" targetFramework="net451" />
+</packages>
\ No newline at end of file


[10/26] lucenenet git commit: Added missing Spatial classes, ported Spatial tests, and fixed several bugs

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Test-Files/Data/states-poly.txt
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Test-Files/Data/states-poly.txt b/src/Lucene.Net.Tests.Spatial/Test-Files/Data/states-poly.txt
new file mode 100644
index 0000000..6164ae0
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Test-Files/Data/states-poly.txt
@@ -0,0 +1,52 @@
+#id	name	shape	
+HI	Hawaii	MULTIPOLYGON (((-160.07380334546815 22.00417734795773, -160.0497093454457 21.988164347942817, -160.0898583454831 21.915870347875487, -160.17013734555786 21.867596347830528, -160.20225934558778 21.795308347763203, -160.24240634562517 21.803280347770627, -160.2263363456102 21.891592347852875, -160.121962345513 21.96397834792029, -160.07380334546815 22.00417734795773)), ((-159.33517434478023 21.94834334790573, -159.43954534487744 21.86807134783097, -159.57602134500453 21.88413634784593, -159.64024734506435 21.94836534790575, -159.7365923451541 21.9644203479207, -159.8008143452139 22.036666347987985, -159.71250634513166 22.14905934809266, -159.57601234500453 22.213179348152376, -159.39136734483256 22.229120348167225, -159.34319534478772 22.197016348137325, -159.29502534474284 22.12481234807008, -159.32713034477274 22.044639347995414, -159.33517434478023 21.94834334790573)), ((-157.67333034323252 21.298027347300074, -157.68137134324002 21.273942347277643, -157.7215013432774 21.
 281971347285122, -157.8258713433746 21.249863347255218, -157.89813434344188 21.330144347329988, -157.94630634348675 21.30606134730756, -158.0988333436288 21.290008347292606, -158.13095134365872 21.354232347352422, -158.23531834375592 21.47465234746457, -158.24334234376337 21.538879347524386, -158.26743134378583 21.58704234756924, -158.11489434364375 21.579016347561765, -158.04262634357644 21.675350347651484, -157.98642834352412 21.699433347673914, -157.91417934345682 21.6352083476141, -157.84995934339702 21.506758347494472, -157.80981534335962 21.43450534742718, -157.76164334331477 21.45858734744961, -157.7134723432699 21.386335347382317, -157.67333034323252 21.298027347300074)), ((-156.71787234234267 21.137419347150498, -156.76604834238753 21.065176347083217, -156.88648434249973 21.049134347068275, -157.0711323426717 21.105331347120615, -157.28789734287355 21.081250347098187, -157.3039483428885 21.137448347150524, -157.24774734283616 21.161530347172953, -157.2316943428212 21.233776
 347240237, -157.16747134276142 21.19364034720286, -157.00690334261185 21.18561034719538, -156.9925463425985 21.195420347204514, -156.958733342567 21.209693347217808, -156.94334634255267 21.172778347183428, -156.91424234252557 21.167372347178393, -156.9102143425218 21.158045347169708, -156.71787234234267 21.137419347150498)), ((-157.03905034264181 20.92870734695612, -156.91058634252215 20.92871834695613, -156.80620534242496 20.840418346873893, -156.81422534243242 20.792252346829034, -156.88648734249972 20.736049346776692, -156.9667653425745 20.728020346769213, -156.99086334259692 20.792237346829022, -156.98282634258945 20.832377346866405, -157.05509834265675 20.88053834691126, -157.03905034264181 20.92870734695612)), ((-156.1960453418567 20.631649346679463, -156.27631734193145 20.583484346634606, -156.3967353420436 20.567427346619652, -156.436879342081 20.623621346671985, -156.46097234210342 20.727981346769177, -156.49307434213333 20.792204346828992, -156.52519334216322 20.7761493468
 14037, -156.63758634226792 20.808261346843945, -156.69378234232025 20.91262434694114, -156.65363634228285 21.016985347038336, -156.59743934223053 21.04106434706076, -156.52519134216323 20.984870347008425, -156.47702234211837 20.896565346926185, -156.35660334200622 20.944726346971038, -156.2602633419165 20.928671346956087, -156.0113923416847 20.800225346836463, -155.9873173416623 20.752061346791606, -156.04351134171463 20.655732346701893, -156.13180934179687 20.623623346671987, -156.1960453418567 20.631649346679463)), ((-155.66619234136323 18.92178634508703, -155.7785853414679 19.01009234516927, -155.85886134154265 19.01010134516928, -155.90703134158753 19.130513345281422, -155.87492734155762 19.371358345505726, -155.94717834162492 19.4837503456104, -155.97125834164734 19.628252345744976, -156.04352434171466 19.78078334588703, -155.97930134165483 19.820922345924416, -155.88295834156511 19.933312346029084, -155.82676134151276 20.005563346096373, -155.83478634152024 20.06175734614871, 
 -155.89097834157258 20.174149346253383, -155.85885834154266 20.270479346343098, -155.73844234143053 20.206256346283283, -155.56182634126603 20.134007346216, -155.20055034092957 19.99752834608889, -155.09618534083236 19.877114345976747, -155.07210634080994 19.724585345834694, -154.97576334072022 19.740643345849648, -154.97576234072022 19.652335345767405, -154.79109634054822 19.53996034566275, -154.81518734057067 19.459676345587976, -154.92758134067535 19.41951834555058, -155.01588134075757 19.331211345468336, -155.1523723408847 19.266992345408525, -155.30489934102675 19.23487834537862, -155.52167834122864 19.122484345273943, -155.5377293412436 19.04221034519918, -155.66619234136323 18.92178634508703)))	
+WA	Washington	MULTIPOLYGON (((-122.40201531038355 48.22521637237797, -122.4628553104402 48.22836337238091, -122.45441931043234 48.128492372287894, -122.36133331034566 48.06009737222419, -122.51451131048832 48.133973372293, -122.54207431051398 48.21046037236424, -122.5091303104833 48.25379337240459, -122.40440431038577 48.24659437239789, -122.37832031036149 48.28972137243805, -122.56436631053475 48.414246372554025, -122.66703231063036 48.41289537255277, -122.69941331066052 48.4943283726286, -122.60817831057555 48.51882437265142, -122.52322831049644 48.45840337259515, -122.47383331045043 48.46219537259868, -122.50529931047973 48.55944437268925, -122.4295453104092 48.59939737272646, -122.48779831046343 48.63857037276294, -122.52655831049954 48.71172437283107, -122.51685331049049 48.757921372874094, -122.69740431065864 48.80301537291609, -122.75424231071159 48.90998837301572, -122.82242131077507 48.95072537305366, -122.74394031070199 48.95580837305839, -122.76511931072172 48.99974637309
 931, -120.85705930894468 48.99983037309939, -118.84360330706951 48.999898373099455, -118.20035430647044 48.999908373099466, -117.43858030576098 48.999918373099476, -117.03204930538237 48.999931373099486, -117.02911130537964 48.838075372948744, -117.03886830538872 48.04618637221124, -117.03747230538742 47.9710923721413, -117.04179430539145 47.36144137157352, -117.04239230539201 47.25850137147765, -117.04096830539069 47.11931937134803, -117.04192630539157 46.53660137080533, -117.03855830538843 46.42798037070417, -117.04447030539394 46.38857437066747, -117.06418430541231 46.34869837063033, -117.02797330537858 46.33542737061797, -117.00164230535405 46.30244837058726, -116.97272530532712 46.24930937053777, -116.96749030532226 46.19755437048957, -116.9294263052868 46.1654833704597, -116.9616373053168 46.09727437039618, -116.98721130534062 46.0785093703787, -116.95772330531315 46.06568737036676, -116.91913230527722 45.995175370301084, -117.4816633058011 45.99983437030543, -117.602826305913
 95 46.00026837030583, -117.98267730626772 45.99988037030547, -117.99252730627688 46.0016393703071, -118.98213330719852 45.99905837030471, -119.03222130724518 45.96627437027417, -119.14025030734578 45.92570837023639, -119.17874230738164 45.922351370233265, -119.30276330749714 45.932662370242866, -119.37944130756856 45.91761037022885, -119.4388613076239 45.91426837022574, -119.51222030769222 45.8992003702117, -119.58929430776399 45.91331537022485, -119.62211630779456 45.899410370211896, -119.67844530784703 45.852539370168245, -119.83355630799147 45.84160937015807, -119.86973530802518 45.83169837014884, -119.9943203081412 45.81114037012969, -120.06864830821043 45.78020237010088, -120.1559083082917 45.76126137008324, -120.20744530833969 45.71978437004461, -120.28363530841065 45.71658337004163, -120.44338330855942 45.6892793700162, -120.49915630861136 45.695630370022116, -120.57008230867743 45.74091837006429, -120.62375730872742 45.743610370066804, -120.65840330875969 45.73261237005656, 
 -120.69699430879562 45.71050937003597, -120.86141930894875 45.66518636999376, -120.90793730899208 45.6354773699661, -120.94857330902992 45.65031636997991, -120.96847830904846 45.6451543699751, -121.033482309109 45.65284436998227, -121.07353030914629 45.64661036997646, -121.12520430919443 45.60705936993963, -121.17431630924017 45.60051636993353, -121.19205430925669 45.61324236994538, -121.20330830926716 45.657287369986406, -121.21427130927736 45.66564536999419, -121.27639130933522 45.67834037000601, -121.31997730937582 45.696642370023056, -121.36781430942037 45.699686370025894, -121.42202930947087 45.690603370017435, -121.44255230948997 45.6949673700215, -121.52905430957054 45.71956737004441, -121.70641730973571 45.688793370015745, -121.7586943097844 45.68971637001661, -121.81104130983316 45.70068337002682, -121.88828330990509 45.67685637000463, -121.92682030994098 45.642028369972195, -121.97265930998367 45.63577636996637, -122.00001131000914 45.61782436994965, -122.08203731008555 45
 .590504369924204, -122.24492231023724 45.54811236988473, -122.30315031029147 45.54309236988006, -122.35645731034111 45.56617136990155, -122.43715431041628 45.56477936990025, -122.56542931053573 45.59481836992823, -122.65120931061563 45.606830369939416, -122.69632331065765 45.63104536996197, -122.76054131071746 45.649397369979056, -122.77255131072863 45.72768537005197, -122.76428831072094 45.760568370082595, -122.78800931074304 45.800343370119634, -122.78451631073978 45.8504493701663, -122.78407331073936 45.867886370182546, -122.80622331076 45.90407237021624, -122.8077413107614 45.94389037025333, -122.87541731082445 46.0271833703309, -122.89975731084711 46.07932937037946, -122.97416931091641 46.110483370408474, -123.05059631098759 46.155736370450626, -123.11855431105087 46.17931037047258, -123.17619631110456 46.18358637047656, -123.2124373111383 46.17000637046391, -123.24879931117218 46.14402037043971, -123.30471731122425 46.14473737044038, -123.4707733113789 46.27502337056171, -123.
 62007631151795 46.25866537054648, -123.7254593116161 46.28542337057141, -123.88577131176541 46.2404383705295, -123.99332931186558 46.31027437059455, -124.07910731194546 46.267259370554484, -124.0655103119328 46.639745370901394, -124.02304331189325 46.58354137084905, -124.0130023118839 46.383680370662915, -123.84145131172413 46.404343370682156, -123.94069331181655 46.481115370753656, -123.89356731177267 46.51107937078156, -123.95771231183241 46.61722537088042, -123.92647031180331 46.67306037093242, -123.84096631172368 46.71828837097454, -123.8955423117745 46.7449863709994, -124.04315831191198 46.71585537097228, -124.09104931195658 46.72902237098454, -124.10206731196683 46.78946937104083, -124.13882731200107 46.89998537114376, -124.10576031197029 46.908148371151356, -124.10473831196933 46.874145371119695, -124.02880831189862 46.823767371072776, -124.04692931191549 46.887253371131905, -123.81265531169731 46.963965371203344, -123.99586431186793 46.97638537121491, -124.03439431190381 47.
 031033371265806, -124.11236131197643 47.042675371276644, -124.16203631202269 46.92961237117135, -124.19273331205127 47.16698237139242, -124.23142531208731 47.275070371493086, -124.31942731216927 47.34923837156216, -124.34908031219689 47.526910371727624, -124.37360531221972 47.6387633718318, -124.48403531232259 47.80825537198965, -124.60668531243681 47.87373537205063, -124.73276931255424 48.149989372307914, -124.70520931252857 48.23199637238429, -124.71717531253971 48.37755737251986, -124.56354731239662 48.35727837250097, -123.99121531186361 48.15916137231646, -123.39685731131007 48.11103037227163, -123.12322231105523 48.14873337230675, -122.92159431086745 48.09417937225594, -122.92484431087047 48.066796372230435, -122.84111131079248 48.13313637229222, -122.76888231072522 48.14399437230233, -122.80293131075693 48.08532137224769, -122.66156031062528 47.91715737209107, -122.65358531061784 47.86443137204197, -122.7458703107038 47.80898837199034, -122.7898013107447 47.80254837198434, -12
 2.80951731076306 47.85707537203512, -122.85880431080896 47.827328372007415, -122.89936331084674 47.672517371863236, -122.9827443109244 47.605474371800796, -123.11391531104655 47.45627337166184, -123.15406031108395 47.348547371561516, -123.01047131095021 47.35302737156569, -122.83324731078517 47.43846437164525, -123.03620631097418 47.3560513715685, -123.11268531104541 47.37156937158295, -123.026336310965 47.51593637171741, -122.91696931086314 47.6146063718093, -122.75294231071038 47.66068837185222, -122.72306231068254 47.75689937194182, -122.61116231057832 47.85000837202854, -122.61321731058024 47.9361893721088, -122.5318883105045 47.90946137208391, -122.4735883104502 47.754980371940036, -122.62150931058797 47.69696837188601, -122.58646031055532 47.57119137176887, -122.55526231052627 47.58350537178033, -122.54270231051457 47.52273437172374, -122.50446131047896 47.50721637170929, -122.55844631052923 47.398363371607914, -122.5441253105159 47.373927371585154, -122.588254310557 47.333929
 371547896, -122.55315631052432 47.28333237150078, -122.5805303105498 47.251387371471026, -122.61154631057869 47.29339837151015, -122.60691431057438 47.2705713714889, -122.69974431066083 47.29208537150893, -122.62875431059472 47.39855337160809, -122.6374363106028 47.39858037160811, -122.74154931069975 47.341450371554906, -122.76970831072599 47.26615637148478, -122.71980131067951 47.22313137144471, -122.7612383107181 47.162496371388244, -122.82510831077758 47.234826371455604, -122.77333531072937 47.3373603715511, -122.80218431075623 47.360740371572874, -122.88037331082904 47.299233371515584, -123.11543631104797 47.207981371430606, -123.08120031101609 47.09005837132078, -123.03134831096966 47.10077437133076, -122.92315031086889 47.04796337128157, -122.79004831074494 47.125859371354125, -122.72818631068732 47.082441371313685, -122.70007931066114 47.09832537132848, -122.5918063105603 47.18006037140459, -122.53076331050346 47.28745637150462, -122.54658831051819 47.31627637153146, -122.424
 09431040412 47.25947237147855, -122.39284331037501 47.27772237149556, -122.44160431042042 47.301125371517344, -122.42083731040108 47.31884437153385, -122.32537631031218 47.34432337155758, -122.31973831030692 47.39011537160023, -122.3926333103748 47.5102423717121, -122.38222031036511 47.59540937179142, -122.41481531039547 47.664180371855466, -122.39449231037653 47.77417637195791, -122.30292231029125 47.95021437212186, -122.23012131022347 47.96911337213946, -122.21699231021123 48.007439372175156, -122.36833331035217 48.12814137228757, -122.40201531038355 48.22521637237797)), ((-122.96797831091064 48.44379437258154, -123.09523331102915 48.47942237261472, -123.15972031108922 48.521842372654234, -123.1698993110987 48.56256437269215, -123.14105331107183 48.62364737274905, -123.10372131103706 48.60837737273482, -123.01209531095174 48.557477372687416, -123.00869831094857 48.53371937266529, -122.96798031091063 48.526933372658974, -123.0222713109612 48.513359372646335, -123.01888331095805 48.
 48960537262421, -122.96797831091064 48.44379437258154)), ((-122.73318731069197 48.27664737242587, -122.66561231062904 48.396777372537755, -122.60438431057202 48.40478937254522, -122.52575031049878 48.32104337246722, -122.52864831050148 48.28351037243227, -122.62350931058984 48.29635037244422, -122.73203431069089 48.22541437237816, -122.61092531057811 48.20632137236038, -122.54620231051783 48.07685837223981, -122.49621231047126 48.09407137225584, -122.37999431036303 48.03214637219817, -122.35540031034013 47.963886372134596, -122.38696131036953 47.90454937207933, -122.44278831042152 47.91805637209191, -122.47161631044837 47.987509372156595, -122.54496131051667 47.967531372137984, -122.60862831057597 48.0314303721975, -122.69555431065692 48.18118537233697, -122.76877831072511 48.21881837237201, -122.73318731069197 48.27664737242587)))	
+MT	Montana	MULTIPOLYGON (((-111.47542530020736 44.702162369096875, -111.48080430021237 44.691416369086866, -111.46069230019364 44.67002336906694, -111.45826530019139 44.652555369050674, -111.47016830020247 44.640710369039645, -111.50769030023741 44.63768836903683, -111.50174730023188 44.615971369016606, -111.51452630024379 44.59319736899539, -111.49290430022364 44.55118936895627, -111.46282730019563 44.549942368955115, -111.45932530019238 44.53792136894391, -111.48257330021403 44.53614336894226, -111.49024130022117 44.528697368935326, -111.56723130029286 44.55286636895784, -111.60524830032827 44.54298936894864, -111.68486230040241 44.55075236895587, -111.71699730043235 44.53376036894004, -111.76691830047884 44.51882536892613, -111.79260830050276 44.518462368925796, -111.80783730051695 44.503982368912304, -111.87250230057717 44.556265368961, -111.9403863006404 44.54972636895491, -111.97781830067525 44.52967636893624, -112.02361330071791 44.53504336894123, -112.02707730072113 44.52284
 336892987, -112.0593673007512 44.528611368935245, -112.09989730078895 44.518231368925576, -112.12419030081158 44.52825336893491, -112.19965830088186 44.531449368937885, -112.21776330089872 44.53849536894445, -112.2303983009105 44.559491368964004, -112.25667530093496 44.55997236896445, -112.28234130095886 44.54170236894744, -112.3425073010149 44.52510036893197, -112.3405773010131 44.49718036890597, -112.36758330103825 44.44927036886135, -112.42075330108777 44.44928436886136, -112.45851930112295 44.46883436887957, -112.50183930116329 44.462997368874134, -112.53932430119819 44.47749736888764, -112.65318930130424 44.48080236889072, -112.71432630136118 44.49693536890574, -112.73371230137924 44.48432036889399, -112.77986330142222 44.47392236888431, -112.79622830143747 44.45801136886949, -112.82669130146583 44.4210843688351, -112.8187103014584 44.394819368810644, -112.81739630145718 44.36420236878213, -112.84427530148221 44.35363936877229, -112.8707813015069 44.36997836878751, -112.8873073
 0152228 44.3928523688088, -112.93828130156976 44.407192368822166, -112.98524930161349 44.435540368848564, -113.01201430163843 44.43771536885059, -113.00665830163344 44.45261536886447, -113.02030930164615 44.48177636889163, -113.00771330163442 44.51061236891848, -113.03782130166246 44.532959368939295, -113.03966030166417 44.55629436896103, -113.08303730170456 44.582681368985604, -113.0542893016778 44.62428936902435, -113.07314430169536 44.67552536907207, -113.0989563017194 44.69591636909106, -113.10170330172195 44.715173369108996, -113.12743130174591 44.73737936912967, -113.13827430175601 44.761439369152086, -113.24033830185107 44.81184136919902, -113.25715430186673 44.81048636919776, -113.31868030192403 44.78022836916958, -113.34063130194447 44.779000369168436, -113.35002430195323 44.80756836919504, -113.42137930201967 44.833699369219374, -113.4455733020422 44.851239369235714, -113.49619130208936 44.930670369309695, -113.48734830208112 44.93957436931798, -113.46341330205883 44.94077
 53693191, -113.44876530204519 44.94952236932725, -113.44102930203798 44.99819436937258, -113.45885330205458 45.02744936939982, -113.4554353020514 45.04334936941463, -113.48630530208014 45.058321369428576, -113.49015930208374 45.071219369440584, -113.52060930211209 45.08206336945069, -113.51022530210243 45.107835369474685, -113.55227230214157 45.10754936947443, -113.57437630216216 45.117711369483885, -113.57158430215956 45.13454536949956, -113.59409930218054 45.14974236951372, -113.6009283021869 45.18099236954282, -113.6455923022285 45.206790369566846, -113.69012030226996 45.26228136961853, -113.68870930226865 45.27778836963297, -113.73908030231556 45.32153036967371, -113.74131030231763 45.38238636973038, -113.77502630234903 45.41017236975626, -113.78566230235894 45.44563336978929, -113.76916830234359 45.47770736981916, -113.7723043023465 45.507054369846486, -113.78093330235454 45.51686536985562, -113.83371530240369 45.514908369853806, -113.8037543023758 45.583729369917904, -113.8224
 8530239323 45.600636369933646, -113.85202730242075 45.609562369941955, -113.90330530246851 45.61349136994562, -113.90219930246748 45.63725336996775, -113.92353230248735 45.65512436998439, -113.9266983024903 45.67121136999937, -113.96414430252517 45.67937837000698, -113.97114930253169 45.69737637002374, -114.00947230256737 45.68633237001346, -114.01987830257707 45.67237837000046, -114.0109903025688 45.65251136998196, -114.01803230257535 45.64077336997103, -114.05651530261119 45.62514436995647, -114.08296730263582 45.58637836992037, -114.11813930266858 45.571127369906165, -114.13204830268154 45.55038236988685, -114.17266730271938 45.54392436988083, -114.19480830273999 45.52791736986592, -114.24199830278394 45.53529036987278, -114.24788030278941 45.502945369842664, -114.26223930280278 45.485859369826755, -114.32643430286257 45.457424369800265, -114.35024630288476 45.46338336980582, -114.3714573029045 45.485740369826644, -114.41905130294883 45.499008369839, -114.43355530296233 45.527633
 369865654, -114.46270830298948 45.54784736988448, -114.49659130302105 45.54664936988337, -114.52739230304974 45.55819336989411, -114.56092430308095 45.54874036988531, -114.54095830306237 45.5963973699297, -114.56467830308446 45.62427136995566, -114.50174130302585 45.65239336998185, -114.5107063030342 45.674057370002025, -114.49756130302194 45.69440137002097, -114.5349763030568 45.7229963700476, -114.5419583030633 45.74599937006903, -114.56354230308341 45.7623983700843, -114.51737530304041 45.81006737012869, -114.49916430302343 45.842683370159065, -114.47380330299983 45.83946837015607, -114.44323130297136 45.85262137016832, -114.4075253029381 45.84645337016258, -114.39283830292442 45.87088637018533, -114.41353030294368 45.91065137022237, -114.42946030295852 45.92147737023245, -114.40529030293601 45.95397937026272, -114.41244730294268 45.97197337027948, -114.48445530300975 45.98980637029609, -114.47452930300051 46.009765370314675, -114.49432130301894 46.02341037032738, -114.4657563029
 9233 46.05081537035291, -114.45602930298327 46.082229370382166, -114.47737030300314 46.107357370405566, -114.50656830303033 46.11614237041375, -114.51894430304186 46.136063370432296, -114.50961330303318 46.15741737045219, -114.4670183029935 46.15526237045018, -114.44087930296917 46.168969370462946, -114.43955330296792 46.22025437051071, -114.47283330299892 46.243783370532626, -114.47379530299982 46.25296137054117, -114.4317953029607 46.28471137057074, -114.40979630294021 46.39291137067151, -114.3970173029283 46.399545370677686, -114.38402530291621 46.428179370704356, -114.41071530294107 46.48737137075948, -114.36046830289428 46.50612537077694, -114.35011530288463 46.51738937078744, -114.3433193028783 46.58788137085309, -114.32471230286097 46.62283937088564, -114.33468530287027 46.65422737091488, -114.3840173029162 46.66159637092174, -114.44153630296978 46.645715370906956, -114.48471830301 46.62357437088633, -114.54039130306184 46.637891370899666, -114.61082630312744 46.6290483708914
 25, -114.64474030315901 46.66082437092102, -114.6450383031593 46.67092137093043, -114.6259263031415 46.6871073709455, -114.67388730318616 46.734721370989845, -114.69843130320902 46.733760370988946, -114.74810530325529 46.695132370952976, -114.78291930328771 46.70304037096034, -114.77783230328296 46.755717371009396, -114.79403030329806 46.76653137101947, -114.8407923033416 46.77553837102786, -114.86660330336565 46.797045371047886, -114.90232530339892 46.799433371050114, -114.94840930344184 46.85244637109949, -114.94056630343454 46.89088837113529, -114.92412530341922 46.90716537115044, -114.96473030345703 46.92521337116725, -115.00157430349134 46.95880937119854, -115.03733430352466 46.963001371202445, -115.0556383035417 46.97335837121209, -115.08133630356564 47.02652437126161, -115.13550730361608 47.06355037129609, -115.14868430362836 47.09174237132235, -115.17249630365053 47.09757037132778, -115.1930733036697 47.124026371352414, -115.29623430376577 47.17955037140412, -115.32522830379
 278 47.245150371465215, -115.34366130380994 47.25502237147441, -115.40820730387006 47.26359337148239, -115.42664130388722 47.274374371492435, -115.50193030395735 47.281644371499205, -115.52306430397702 47.29198237150884, -115.55552030400725 47.33461337154854, -115.59953630404824 47.3700033715815, -115.6387823040848 47.38004437159085, -115.66647730411059 47.399167371608655, -115.75032630418868 47.42247537163037, -115.75010530418848 47.43396637164106, -115.73248130417207 47.445303371651626, -115.65608730410091 47.44918037165523, -115.64318530408889 47.45779337166326, -115.64014230408605 47.4752353716795, -115.69277030413508 47.489540371692826, -115.70152230414323 47.520893371722025, -115.7428293041817 47.533691371733944, -115.69208830413444 47.590721371787055, -115.6982843041402 47.616080371810675, -115.73406730417354 47.63987937183283, -115.73366530417316 47.69555437188469, -115.77572730421232 47.70973237189789, -115.79053730422612 47.744838371930584, -115.83674230426917 47.756281371
 94124, -115.84932430428088 47.80518237198679, -115.86980930429996 47.82745237200753, -115.90392130433173 47.841074372020216, -115.93784230436331 47.86712437204447, -115.99893230442021 47.92514037209851, -116.02531630444479 47.964939372135575, -116.05349230447102 47.976191372146054, -116.0554973044729 48.208483372362394, -116.05669230447401 48.498665372632644, -116.06353130448036 48.9999503730995, -114.7293253032378 48.99997037309952, -114.06346330261766 48.999977373099526, -112.18838730087137 48.999992373099545, -111.28267930002785 49.00001137309956, -110.75079729953251 49.000005373099555, -109.50073729836829 49.000005373099555, -108.25067529720408 49.00000937309956, -107.1881212962145 49.00001737309957, -106.12557929522494 49.00002137309957, -105.06303429423536 49.00002137309957, -104.062991293304 49.00002637309957, -104.05231729329405 48.645824372769695, -104.05211129329386 48.39101937253239, -104.04842529329044 48.0000813721683, -104.04730729328939 47.40001737160945, -104.0459262
 932881 47.333832371547814, -104.04743729328952 46.64294737090437, -104.04670529328884 46.54253937081086, -104.04783629328989 46.28088137056717, -104.04890629329088 45.942993370252495, -104.04951729329144 45.883052370196665, -104.04385129328617 45.212875369572515, -104.04307229328545 44.997805369372216, -104.05984229330106 44.99733636937178, -105.04179629421559 45.00107636937526, -105.08500329425583 44.999817369374085, -106.02115029512768 44.997213369371664, -106.25923129534941 44.99616236937068, -107.89437429687226 44.99977336937405, -108.25923829721206 45.00011536937437, -108.62525629755294 44.99759336937201, -109.79938529864643 44.999522369373814, -109.99552929882911 45.00279336937686, -110.39276029919905 44.998625369372974, -110.42964929923342 44.992285369367075, -111.05342829981436 44.99569536937025, -111.05161629981266 44.664490369061795, -111.0515602998126 44.473323368883754, -111.09463029985272 44.48612436889567, -111.12891829988466 44.500757368909305, -111.13435929988972 44.
 52790236893458, -111.17024229992315 44.54518636895068, -111.17876429993107 44.564851368968995, -111.21950729996902 44.57317036897675, -111.23423329998273 44.60256236900412, -111.2197972999693 44.61798136901848, -111.22397129997319 44.62690836902679, -111.27066530001667 44.64221236904105, -111.27020830001624 44.673802369070465, -111.29566830003995 44.68293836907897, -111.3154753000584 44.7051933690997, -111.31922130006188 44.72786436912081, -111.34997730009053 44.72617736911924, -111.37230930011133 44.745087369136854, -111.38495930012311 44.73769436912997, -111.39508430013254 44.70886936910313, -111.44363230017775 44.71317936910714, -111.47542530020736 44.702162369096875)))	
+ME	Maine	MULTIPOLYGON (((-69.77727626137293 44.07414836851199, -69.85992826144991 44.00000136844294, -69.7915282613862 43.75608536821577, -69.8303922614224 43.727986368189605, -69.85178526144233 43.74432836820482, -69.84615526143709 43.84234436829611, -69.88679126147493 43.87671336832811, -69.90313226149014 43.790732368248044, -69.97290326155513 43.768847368227654, -69.9995002615799 43.78620736824382, -69.9873702615686 43.845738368299266, -70.02640326160495 43.84560136829914, -70.15662826172624 43.78981036824718, -70.23579826179997 43.68579636815031, -70.22223926178734 43.57724036804921, -70.34161026189851 43.53490836800978, -70.36592526192116 43.430303367912366, -70.45697726200596 43.349470367837085, -70.53894126208229 43.33571836782427, -70.66567226220032 43.09105036759641, -70.81866826234281 43.12187136762512, -70.83054826235387 43.15917436765986, -70.81320726233773 43.235222367730685, -70.90108626241957 43.28102036777334, -70.90580126242396 43.30206936779294, -70.96969926248347 
 43.36638036785283, -70.97909926249223 43.39618436788059, -70.96148326247581 43.43812636791965, -70.97079126248448 43.47021136794953, -70.95927826247376 43.51638836799253, -70.96426826247841 43.53198936800707, -70.94961926246476 43.548953368022865, -70.95652426247119 43.56414336803701, -70.97387426248736 43.571830368044175, -70.9844422624972 43.79116336824844, -71.00859626251969 44.28214636870571, -71.02872626253844 44.66853836906556, -71.08750926259319 45.301469369655024, -70.95938226247385 45.338865369689856, -70.87644426239662 45.22544536958422, -70.84287526236535 45.2781373696333, -70.81266626233722 45.35467836970458, -70.82913226235256 45.39072636973815, -70.7969672623226 45.42517236977023, -70.63492926217168 45.391967369739305, -70.71991026225083 45.51295436985198, -70.55227026209471 45.66066436998955, -70.39638326194952 45.72204637004671, -70.416214261968 45.79030937011029, -70.25396426181689 45.89900437021152, -70.24746426181083 45.944619370254, -70.31029526186934 45.96878237
 027651, -70.28002226184115 46.05315437035509, -70.30484926186428 46.06665837036766, -70.22932526179395 46.13743437043358, -70.28349626184439 46.19024937048276, -70.1910582617583 46.33483937061742, -70.04660726162378 46.42611537070243, -70.01414426159354 46.57059837083699, -69.98497726156637 46.69136537094947, -69.23029626086353 47.4533343716591, -69.04697626069279 47.42203037162995, -69.03671426068324 47.25736137147659, -68.89487226055114 47.182256371406645, -68.51467326019704 47.296964371513475, -68.3912572600821 47.28509737150242, -68.33481426002953 47.35737437156973, -68.23080725993267 47.352148371564866, -67.79101125952309 47.06100337129372, -67.7802892595131 45.947062370256276, -67.75561525949011 45.91658037022789, -67.79457125952639 45.878475370192405, -67.75936725949362 45.8277983701452, -67.8030532595343 45.7945083701142, -67.80343325953466 45.6781133700058, -67.75295525948763 45.65928936998827, -67.71803425945512 45.68129937000877, -67.6151402593593 45.60519936993789, -67.4
 3930125919553 45.59256136992612, -67.4160842591739 45.50355436984323, -67.50410625925588 45.48581636982671, -67.4185552591762 45.3758523697243, -67.47795025923152 45.28028036963529, -67.43943525919565 45.18958436955082, -67.34560525910827 45.122252369488116, -67.27409525904167 45.18278336954449, -67.16590525894091 45.15626436951979, -67.1506612589267 45.121990369487875, -67.06535825884727 44.959295369336346, -67.14670625892303 44.904581369285395, -66.96927125875777 44.82865536921469, -67.00771925879359 44.780625369169954, -67.200364258973 44.65378136905181, -67.30846825907368 44.653521369051575, -67.38851025914822 44.69140036908685, -67.57099325931817 44.59833336900017, -67.61883825936273 44.540239368946075, -67.8112192595419 44.5540093689589, -67.858560259586 44.5360773689422, -67.90004225962463 44.45239936886426, -67.96834225968823 44.4712283688818, -67.96343625968366 44.505327368913555, -67.98652325970517 44.48481236889445, -68.01639325973298 44.384956368801454, -68.074379259787 
 44.38137436879812, -68.13626425984462 44.47523736888554, -68.24561425994646 44.49064836889988, -68.3637652600565 44.4313863688447, -68.42857126011685 44.465306368876284, -68.55218626023198 44.39904936881458, -68.53007526021139 44.28983636871287, -68.55942726023872 44.25988736868497, -68.74031026040718 44.34633036876548, -68.81285126047474 44.32743236874788, -68.8137682604756 44.41399036882849, -68.74134826040816 44.50728536891538, -68.74527926041182 44.552320368957325, -68.82355226048472 44.60890636901003, -68.82381326048495 44.664089369061415, -68.86060926051923 44.61097036901195, -68.80790326047014 44.56965436897347, -68.81167826047366 44.494593368903566, -68.95917926061102 44.43033136884371, -68.9850282606351 44.27111236869543, -69.02148226066905 44.24409336867026, -69.07445826071839 44.06906636850726, -69.21914026085314 43.94678736839337, -69.29365026092252 43.94219036838909, -69.3464542609717 44.01596936845781, -69.39448826101643 44.02512836846634, -69.48323326109909 43.8871603
 68337845, -69.5893262611979 43.84486236829845, -69.66445326126787 43.85222436830531, -69.65524526125928 43.98025036842454, -69.61293226121988 44.03361236847424, -69.72063526132018 43.93797936838517, -69.74852826134617 43.893375368343634, -69.72467126132395 43.784477368242214, -69.75035926134787 43.761704368221004, -69.77767326137331 43.79127036824854, -69.80001326139411 44.02686636846796, -69.76675526136314 44.04773236848739, -69.77727626137293 44.07414836851199)), ((-68.387921260079 44.377253368794285, -68.35025426004391 44.398951368814494, -68.35544926004876 44.428857368842344, -68.23870925994004 44.43756336885045, -68.16476925987118 44.33449536875446, -68.3047052600015 44.290031368713045, -68.32071126001641 44.22507936865256, -68.40289026009295 44.27080136869514, -68.387921260079 44.377253368794285)))	
+ND	North Dakota	MULTIPOLYGON (((-98.73043728833767 45.93827137024809, -99.00683328859509 45.93955537024929, -99.7173452892568 45.94276137025227, -99.87578328940435 45.94354737025301, -100.51440628999912 45.94038837025006, -102.00277529138528 45.942505370252036, -102.9463972922641 45.94166537025126, -102.99482329230919 45.94111537025074, -104.04890629329088 45.942993370252495, -104.04783629328989 46.28088137056717, -104.04670529328884 46.54253937081086, -104.04743729328952 46.64294737090437, -104.0459262932881 47.333832371547814, -104.04730729328939 47.40001737160945, -104.04842529329044 48.0000813721683, -104.05211129329386 48.39101937253239, -104.05231729329405 48.645824372769695, -104.062991293304 49.00002637309957, -102.93795929225622 49.00002637309957, -102.02226429140342 49.000015373099565, -101.50043729091743 49.000020373099574, -100.18790828969505 49.00000237309955, -99.53356628908564 49.00000837309956, -99.0004032885891 49.00000637309955, -97.93786728759953 48.99999237309954
 5, -97.22943628693976 48.999987373099536, -97.21636928692759 48.931830373036064, -97.17572728688974 48.87375737298198, -97.17120428688553 48.83598037294679, -97.1804222868941 48.81553737292775, -97.16471228687948 48.81036837292294, -97.17394428688807 48.801514372914696, -97.14751628686346 48.781170372895744, -97.13924628685575 48.76354237287933, -97.14789828686382 48.75565337287198, -97.13250228684947 48.747218372864126, -97.13480628685163 48.72623837284459, -97.11010128682861 48.708583372828144, -97.1167392868348 48.695243372815725, -97.09716928681657 48.67452937279643, -97.1076302868263 48.629946372754915, -97.12744428684476 48.62979437275477, -97.12295828684059 48.62076837274636, -97.14471828686085 48.61402437274008, -97.14081228685721 48.586905372714824, -97.1581922868734 48.583640372711784, -97.15212728686775 48.572856372701736, -97.16794328688249 48.56226337269187, -97.14661828686262 48.54953737268002, -97.1604352868755 48.545078372675874, -97.15553728687092 48.53839837266965,
  -97.13938528685588 48.534648372666155, -97.14832728686422 48.51795137265061, -97.13459428685142 48.51731437265001, -97.14361328685983 48.43810937257625, -97.1196332868375 48.43710237257531, -97.12260128684025 48.416110372555764, -97.1516472868673 48.41961237255902, -97.14982328686561 48.40999137255006, -97.12912428684633 48.4078853725481, -97.15881928687399 48.38820637252977, -97.135205286852 48.38441037252623, -97.13378628685068 48.3724543725151, -97.15039628686614 48.3632153725065, -97.1311232868482 48.361491372504894, -97.13713628685379 48.32599137247183, -97.11259128683093 48.319926372466185, -97.1326342868496 48.31096937245784, -97.11475128683294 48.303618372451, -97.11372128683199 48.294882372442856, -97.13051328684763 48.29304037244114, -97.11268328683101 48.286147372434726, -97.11171428683012 48.277876372427016, -97.13665528685334 48.264483372414546, -97.12378428684136 48.259173372409606, -97.12755428684487 48.23352337238571, -97.10923528682781 48.22804937238061, -97.139754
 28685623 48.22175537237475, -97.11089928682935 48.20760537236157, -97.13082828684792 48.20374237235798, -97.13727528685392 48.19506337234989, -97.13629128685301 48.17522737233142, -97.13744328685408 48.16776937232447, -97.11606528683417 48.15922337231652, -97.13651328685322 48.14839837230643, -97.12091828683869 48.1427743723012, -97.12187328683957 48.11636937227661, -97.0990302868183 48.10097237226226, -97.09272128681243 48.07034437223374, -97.06707128678855 48.04816437221308, -97.04805328677082 47.95492437212624, -97.01533128674035 47.917890372091755, -97.02056628674522 47.87556937205234, -97.0003402867264 47.87019737204734, -96.97723128670486 47.82802937200807, -96.98389328671108 47.809661371990956, -96.9578302866868 47.79444037197678, -96.93201228666275 47.763506371947976, -96.92365928665498 47.71409437190196, -96.8894252866231 47.67392537186454, -96.87333528660811 47.61525537180991, -96.85221728658844 47.601151371796774, -96.85866428659445 47.56297837176122, -96.84918828658562 4
 7.54456837174408, -96.86068728659633 47.521356371722455, -96.85161528658789 47.50061937170314, -96.86668428660191 47.46153737166674, -96.8558272865918 47.43675337164366, -96.86724828660245 47.41308737162162, -96.85000528658638 47.408936371617756, -96.8398272865769 47.38411737159464, -96.85063128658696 47.36095437157307, -96.83846128657564 47.34224337155564, -96.84674728658335 47.3146023715299, -96.83771428657494 47.2938843715106, -96.84962328658602 47.25684337147611, -96.83706528657433 47.240458371460846, -96.82649128656448 47.17006337139529, -96.83916428657629 47.15188637137836, -96.81915128655764 47.09260437132315, -96.82696428656492 47.07883237131033, -96.82260828656086 47.033932371268506, -96.83529628657269 47.010231371246434, -96.82453128656266 47.003436371240106, -96.81677228655543 46.96977937120876, -96.79342528653369 46.96964137120863, -96.80188728654157 46.95584337119578, -96.78971028653022 46.948202371188664, -96.78792528652856 46.93218437117375, -96.76306828650542 46.9362
 61371177544, -96.75691128649969 46.92278037116499, -96.77806128651937 46.86734937111336, -96.76825028651024 46.84486137109242, -96.7971972865372 46.812033371061844, -96.78038228652154 46.76231237101554, -96.78155628652263 46.70704437096407, -96.79369528653393 46.67880437093777, -96.79024628653073 46.6297733708921, -96.78431728652521 46.624112370886834, -96.77104128651284 46.59998337086436, -96.75122728649438 46.58861937085378, -96.74031628648423 46.4894323707614, -96.71489428646055 46.46871837074211, -96.70968228645569 46.42716837070341, -96.68822828643572 46.41221837068949, -96.65210128640207 46.35943337064033, -96.61486128636739 46.3508123706323, -96.60207428635547 46.33632437061881, -96.59818328635185 46.23868237052787, -96.58645628634093 46.2154133705062, -96.58789028634227 46.191918370484316, -96.57116628632669 46.17717437047059, -96.55193128630877 46.09552937039455, -96.57621528633139 46.0212793703254, -96.56180228631797 45.947683370256854, -96.56692128632274 45.93411037024421
 , -97.23331028694336 45.936502370246444, -97.97872228763758 45.93082237024116, -98.0147092876711 45.93149837024178, -98.73043728833767 45.93827137024809)))	
+SD	South Dakota	MULTIPOLYGON (((-102.78838429211693 42.99530336750724, -103.00587529231949 42.99935436751102, -103.50146429278104 42.99861836751033, -104.05619929329767 43.00306236751447, -104.05915729330043 43.47913436795784, -104.05791429329928 43.50371236798073, -104.05947929330073 43.852906368305945, -104.05973129330096 44.14582536857875, -104.06103629330218 44.18182536861227, -104.05946529330072 44.57435236897784, -104.05984229330106 44.99733636937178, -104.04307229328545 44.997805369372216, -104.04385129328617 45.212875369572515, -104.04951729329144 45.883052370196665, -104.04890629329088 45.942993370252495, -102.99482329230919 45.94111537025074, -102.9463972922641 45.94166537025126, -102.00277529138528 45.942505370252036, -100.51440628999912 45.94038837025006, -99.87578328940435 45.94354737025301, -99.7173452892568 45.94276137025227, -99.00683328859509 45.93955537024929, -98.73043728833767 45.93827137024809, -98.0147092876711 45.93149837024178, -97.97872228763758 45.930822370
 24116, -97.23331028694336 45.936502370246444, -96.56692128632274 45.93411037024421, -96.58795528634232 45.81785437013595, -96.60461028635784 45.80826437012701, -96.65739128640699 45.738970370062475, -96.83279628657036 45.65068736998026, -96.85499028659102 45.609122369941545, -96.84308728657994 45.584090369918236, -96.76924628651116 45.5174783698562, -96.7380322864821 45.45819536980099, -96.69316928644032 45.4106383697567, -96.60508428635828 45.39652436974355, -96.53254928629073 45.37513236972363, -96.47759228623954 45.328509369680205, -96.45760228622093 45.298850369652584, -96.45449628621803 45.27519536963055, -96.4560802862195 44.97199436934818, -96.45521728621871 44.801347369189244, -96.45671828622011 44.62880836902856, -96.45510628621861 44.53834336894431, -96.45739728622074 44.19906136862833, -96.45660228621999 43.848741368302065, -96.46045428622358 43.49971836797701, -96.59831528635198 43.499849367977134, -96.58379628633845 43.48192036796044, -96.5891132863434 43.43553936791724
 4, -96.55770828631415 43.40072736788482, -96.52505328628375 43.38422536786945, -96.52289428628174 43.356966367844066, -96.5405632862982 43.307659367798145, -96.5791312863341 43.29007436778177, -96.57072228632627 43.26361236775712, -96.5595672863159 43.25326336774748, -96.5669912863228 43.23963336773479, -96.558605286315 43.22548936772162, -96.48724528624854 43.217909367714554, -96.47311428623537 43.20908236770634, -96.45150528621525 43.12630836762925, -96.46080528622392 43.08787236759345, -96.46209428622511 43.075582367582, -96.47957328624139 43.06188436756925, -96.52001028627905 43.051508367559585, -96.4990202862595 43.01205036752283, -96.51714828627638 42.986458367499, -96.51493528627432 42.952382367467266, -96.54426328630164 42.913866367431396, -96.53751128629536 42.896906367415596, -96.55621128631276 42.846660367368806, -96.57312628632852 42.83434736735734, -96.58764528634204 42.8353813673583, -96.60087528635437 42.799558367324934, -96.63298028638427 42.776835367303775, -96.6407
 0928639146 42.74860336727748, -96.62654028637826 42.70835436724, -96.56303928631912 42.66851336720289, -96.54116528629875 42.66240536719721, -96.51284428627237 42.629755367166794, -96.48849828624971 42.580480367120906, -96.50094228626129 42.57388536711476, -96.48933728625049 42.56402836710558, -96.48024328624201 42.51713036706191, -96.43939428620396 42.48924036703593, -96.49470128625548 42.488459367035205, -96.5472152863044 42.52049936706504, -96.58475328633935 42.51828736706298, -96.60546728635863 42.50723636705269, -96.62929428638083 42.52269336706709, -96.6366722863877 42.5507313670932, -96.71405928645977 42.61230236715054, -96.7152732864609 42.62190736715949, -96.69459628644165 42.64116336717742, -96.6990602864458 42.657715367192836, -96.72265828646778 42.66859236720296, -96.7993442865392 42.67001936720429, -96.81043728654953 42.68134136721484, -96.81014028654926 42.70408436723602, -96.9082342866406 42.73169936726174, -96.97077328669886 42.721147367251916, -96.97786928670547 42.
 72730836725765, -96.97000328669814 42.75206536728071, -96.97959328670707 42.758313367286526, -97.01513928674018 42.759542367287665, -97.13046928684759 42.773923367301066, -97.16142228687642 42.798619367324065, -97.21183128692336 42.81257336733706, -97.2244432869351 42.84120236736372, -97.24318928695256 42.85182636737362, -97.27145728697889 42.85001436737193, -97.3114142870161 42.86177136738288, -97.38930628708864 42.86743336738815, -97.45726328715193 42.85044336737233, -97.48315928717605 42.857157367378576, -97.50613228719745 42.86013636738136, -97.57065428725754 42.847990367370045, -97.63497028731744 42.86128536738242, -97.68575228736474 42.83683736735966, -97.72525028740152 42.85800836737937, -97.77218628744522 42.846164367368345, -97.79702828746836 42.849597367371544, -97.8186432874885 42.86658736738737, -97.88865928755371 42.855807367377324, -97.88994128755489 42.831271367354475, -97.92947728759172 42.7923243673182, -97.96355828762346 42.773690367300844, -97.99514428765288 42.76
 681236729444, -98.03314028768825 42.769192367296654, -98.12182028777084 42.80836036733314, -98.12311728777206 42.820223367344184, -98.14486928779232 42.83579436735869, -98.1678262878137 42.839571367362204, -98.31033928794642 42.881794367401525, -98.39120428802174 42.92013536743723, -98.45744428808342 42.93716036745309, -98.49765128812088 42.991778367503954, -99.25397128882526 42.99238936750453, -99.53279028908491 42.992335367504474, -100.19814228970458 42.99109536750332, -101.23173729066718 42.98684336749936, -102.08670129146344 42.98988736750219, -102.78838429211693 42.99530336750724)))	
+WY	Wyoming	MULTIPOLYGON (((-104.05361529329527 41.69821836629923, -104.05550029329702 41.56422236617444, -104.05401229329564 41.3880853660104, -104.0517052932935 41.003211365651964, -104.93449329411565 40.99428936564365, -105.2787972944363 40.99634936564557, -106.20347129529748 41.00008536564905, -106.3291252954145 41.001289365650166, -106.86543829591398 40.99845736564753, -107.30405129632247 41.00013336564909, -107.91867129689489 41.00337536565211, -109.04831429794694 40.998433365647514, -110.00216529883528 40.997599365646735, -110.06318529889211 40.997892365647004, -111.0510222998121 40.99658336564579, -111.05165129981269 41.25842536588965, -111.05106829981216 41.57859236618782, -111.04869729980994 41.99620336657675, -111.04678029980816 42.50325236704899, -111.04921529981043 43.01988336753013, -111.04749829980882 43.28473436777679, -111.04677129980816 43.515528367991735, -111.05040529981153 43.98255336842669, -111.0515602998126 44.473323368883754, -111.05161629981266 44.6644903690
 61795, -111.05342829981436 44.99569536937025, -110.42964929923342 44.992285369367075, -110.39276029919905 44.998625369372974, -109.99552929882911 45.00279336937686, -109.79938529864643 44.999522369373814, -108.62525629755294 44.99759336937201, -108.25923829721206 45.00011536937437, -107.89437429687226 44.99977336937405, -106.25923129534941 44.99616236937068, -106.02115029512768 44.997213369371664, -105.08500329425583 44.999817369374085, -105.04179629421559 45.00107636937526, -104.05984229330106 44.99733636937178, -104.05946529330072 44.57435236897784, -104.06103629330218 44.18182536861227, -104.05973129330096 44.14582536857875, -104.05947929330073 43.852906368305945, -104.05791429329928 43.50371236798073, -104.05915729330043 43.47913436795784, -104.05619929329767 43.00306236751447, -104.05621929329769 42.61466936715274, -104.05351329329517 41.99981536658012, -104.05361529329527 41.69821836629923)))	
+WI	Wisconsin	MULTIPOLYGON (((-87.74855527810999 44.961616369338515, -87.83999227819515 44.92732336930658, -87.8310202781868 44.8733463692563, -87.98579127833094 44.72047436911393, -87.98318227832851 44.67726536907369, -88.01328827835654 44.63911836903816, -87.97575827832159 44.595814368997836, -88.0130212783563 44.622234369022436, -88.04041727838181 44.57144936897514, -87.96622827831271 44.53549636894166, -87.92640827827563 44.539139368945044, -87.86878227822197 44.61690636901747, -87.7642262781246 44.64404836904275, -87.72382127808696 44.68928736908488, -87.6144642779851 44.833047369218775, -87.55278727792768 44.851335369235805, -87.55167227792663 44.82302336920944, -87.4337472778168 44.89109636927283, -87.36745927775507 44.81156736919877, -87.31446527770572 44.79471836918307, -87.37307027776029 44.67691836907336, -87.47352827785386 44.533946368940214, -87.53748927791342 44.32785136874827, -87.51732227789465 44.17575436860662, -87.64437027801297 44.09783036853405, -87.7261222780891
 1 43.893904368344124, -87.70273027806732 43.673176368138556, -87.78604527814491 43.54629736802039, -87.80295927816066 43.458714367938825, -87.87533227822807 43.358592367845574, -87.88983427824157 43.19721636769529, -87.86006927821386 43.07587536758228, -87.89198327824357 43.02577436753562, -87.83643827819185 42.96459236747864, -87.81984927817639 42.84156336736406, -87.75680327811767 42.77754636730444, -87.79150927815 42.66664236720115, -87.79738227815547 42.48915236703585, -88.19479027852559 42.489631367036296, -88.2979892786217 42.49198836703849, -88.70662327900227 42.48967136703634, -88.76505827905669 42.4909223670375, -88.93918727921886 42.49087936703746, -89.35955927961037 42.49791736704401, -89.4006132796486 42.49750236704362, -89.8347392800529 42.50346836704918, -89.92369128013574 42.504115367049785, -90.42010328059807 42.50836536705374, -90.63845628080142 42.509363367054675, -90.62570728078956 42.52856236707255, -90.63921928080214 42.55571436709784, -90.66438028082557 42.5713
 9136711244, -90.69479128085389 42.63792836717441, -90.74561028090122 42.65700136719217, -90.89254528103807 42.678240367211956, -90.91940928106308 42.68067736721422, -90.99918228113738 42.70705836723879, -91.06616828119977 42.744913367274044, -91.08203028121454 42.783365367309855, -91.09342828122514 42.871440367391884, -91.1391212812677 42.9258933674426, -91.1522142812799 43.00131636751284, -91.15975228128691 43.08118236758722, -91.16857128129513 43.08288836758881, -91.16135428128841 43.14757636764905, -91.06905228120245 43.2578983677518, -91.0664282812 43.28068336777302, -91.07849828121125 43.31329736780339, -91.17704828130303 43.35394636784125, -91.19824328132277 43.37051336785668, -91.21091628133458 43.42405136790654, -91.23590328135784 43.464684367944386, -91.22356628134635 43.500808367978024, -91.24055828136218 43.54871236802264, -91.23299028135513 43.59889036806938, -91.25838928137878 43.67732236814242, -91.25891628137927 43.722395368184394, -91.251105281372 43.788075368245565,
  -91.29194828141004 43.847190368300616, -91.37335728148585 43.94719136839375, -91.42590228153479 43.98561936842954, -91.52842028163028 44.034215368474804, -91.56916228166821 44.034955368475494, -91.6017862816986 44.04082236848096, -91.65223328174558 44.066895368505236, -91.75321928183963 44.137227368570734, -91.84874428192859 44.191187368620994, -91.8886942819658 44.25749536868275, -91.92234928199714 44.28834136871147, -91.92275428199753 44.31752036873865, -91.93886828201254 44.33911136875876, -91.97238628204374 44.36448736878239, -92.09133328215452 44.415589368829984, -92.20613728226144 44.43839436885122, -92.24910028230146 44.45621636886782, -92.29668728234577 44.49218236890132, -92.32047828236793 44.540491368946306, -92.34087228238693 44.5528353689578, -92.5092152825437 44.575159368978596, -92.60897328263661 44.61029236901132, -92.63036728265654 44.64265236904146, -92.73714528275598 44.713594369107525, -92.80558428281972 44.746160369137854, -92.76102828277823 44.83537136922094, -
 92.76426328278124 44.862234369245954, -92.77187128278833 44.899496369280655, -92.75392628277162 44.915002369295095, -92.74976828276773 44.93565536931433, -92.7671262827839 45.00100536937519, -92.76299128278005 45.02211936939486, -92.7967622828115 45.06561036943536, -92.74542228276368 45.1130043694795, -92.74493528276324 45.15642236951994, -92.76258328277967 45.18661236954806, -92.755419282773 45.21237636957205, -92.74659328276478 45.297603369651426, -92.70738428272827 45.318201369670604, -92.6848692827073 45.3630763697124, -92.64875128267366 45.395466369742564, -92.64497528267015 45.43945236978353, -92.6548172826793 45.45522136979822, -92.68542128270781 45.470053369812035, -92.72815428274761 45.54724236988392, -92.7621752827793 45.56426336989977, -92.83503728284715 45.56340236989897, -92.87683128288607 45.57883636991335, -92.88539728289405 45.64495536997492, -92.86001928287041 45.71056237003602, -92.83363628284584 45.73089037005495, -92.77910728279507 45.763340370085174, -92.7487622
 827668 45.837302370154056, -92.73409728275314 45.84498037016121, -92.7062402827272 45.89095837020403, -92.66620828268992 45.91570337022708, -92.55267228258418 45.9512693702602, -92.52397728255745 45.98258337028936, -92.46234528250005 45.98119737028807, -92.42499928246528 46.02550437032933, -92.36496328240936 46.01624837032071, -92.34622528239191 46.022596370326625, -92.32737228237436 46.056878370358554, -92.28937028233896 46.07323137037378, -92.28894428233856 46.15660037045143, -92.28868528233832 46.415984370692996, -92.287271282337 46.658786370919124, -92.20915428226425 46.64687237090803, -92.09597028215885 46.74262737099721, -92.00415728207334 46.68380037094242, -91.92146128199632 46.680134370939, -91.55577328165575 46.75686037101046, -90.86173028100937 46.95247937119265, -90.77448628092812 46.92023537116262, -90.77744528093086 46.88312237112805, -90.92624428106944 46.58550337085087, -90.73071428088734 46.64569637090693, -90.54087728071055 46.58752637085276, -90.40820028058698 46.
 568610370835145, -90.38552528056586 46.53965737080817, -90.31370828049899 46.55156337081927, -90.30239328048845 46.544296370812496, -90.30018128048638 46.52505137079457, -90.26978528045808 46.52248037079218, -90.25840128044747 46.508789370779425, -90.21152628040382 46.50629537077711, -90.16139128035712 46.44238037071758, -90.14179728033888 46.39389937067243, -90.11517728031409 46.36515537064566, -90.1116592803108 46.34042937062263, -89.9251362801371 46.30402537058873, -89.09980627936845 46.14564237044122, -88.9853012792618 46.10039137039908, -88.92519527920582 46.07360137037413, -88.80439727909332 46.026804370330545, -88.79381527908347 46.036360370339445, -88.77748027906826 46.032614370335956, -88.7730172790641 46.02114737032528, -88.72640927902069 46.02958137033313, -88.70360527899946 46.01892337032321, -88.67738427897504 46.020144370324346, -88.64366927894363 45.99338837029943, -88.6155022789174 45.99412037030011, -88.59753627890068 46.01551637032003, -88.57535727888002 46.0089593
 70313924, -88.54835827885486 46.019300370323556, -88.51561327882438 46.01860937032291, -88.49408327880433 46.01296037031766, -88.48381427879475 45.99915137030479, -88.45431927876729 46.00076037030629, -88.40352227871998 45.98342237029014, -88.3699382786887 45.994587370300536, -88.32132327864343 45.96671237027458, -88.29915227862278 45.96194437027014, -88.25716827858368 45.9670553702749, -88.2149922785444 45.94790137025706, -88.18019427851199 45.95351637026229, -88.15043827848427 45.93629337024625, -88.11139027844791 45.926287370236935, -88.09385027843157 45.920615370231644, -88.09576427843336 45.89180337020481, -88.0654212784051 45.8736423701879, -88.12178627845759 45.8348783701518, -88.12994927846519 45.81940237013738, -88.08873427842681 45.791532370111426, -88.05163927839226 45.78611237010638, -87.99007027833493 45.795046370114704, -87.96917927831547 45.76644837008807, -87.87362927822647 45.750699370073406, -87.84236227819736 45.722418370047066, -87.80155327815935 45.7113913700367
 9, -87.80115627815898 45.70132437002742, -87.77747327813692 45.684101370011376, -87.78094527814017 45.67591537000375, -87.81705427817379 45.66539036999395, -87.81993827817648 45.65445036998376, -87.7760452781356 45.613200369945346, -87.7750752781347 45.600387369933415, -87.78631227814516 45.56851936990373, -87.82860227818455 45.5685913699038, -87.8051412781627 45.544525369881384, -87.78938527814802 45.499067369839054, -87.81361427817059 45.466460369808686, -87.86026727821404 45.44509836978879, -87.84953127820404 45.40611736975249, -87.88361027823578 45.36585436971499, -87.8739742782268 45.36208536971148, -87.86853527822174 45.37207236972078, -87.86209627821574 45.370165369719004, -87.84128227819636 45.34614936969663, -87.82800827818399 45.35832136970797, -87.76003827812069 45.352897369702916, -87.68959827805509 45.39126936973865, -87.64368427801233 45.36185636971126, -87.64536227801389 45.34816936969852, -87.70447127806894 45.27220536962777, -87.70514227806956 45.247086369604375, -8
 7.71966827808309 45.23677136959477, -87.72162827808492 45.211672369571396, -87.7362002780985 45.19907236955966, -87.7296692780924 45.17660436953874, -87.67281427803945 45.14067236950527, -87.66488627803207 45.10905436947583, -87.5812762779542 45.0946403694624, -87.61852127798889 45.05680736942716, -87.62033527799058 44.99199736936681, -87.74855527810999 44.961616369338515)), ((-87.034524277445 45.290405369644716, -86.98625327740005 45.2986573696524, -86.96771227738277 45.24027736959803, -86.99573427740887 45.218411369577666, -87.04511227745486 45.249019369606174, -87.02544827743655 45.149974369513934, -87.07987627748724 45.14730736951145, -87.04490127745467 45.09551336946321, -87.0876782774945 45.09217836946011, -87.08390027749098 45.05328536942389, -87.11255727751768 45.06476336943457, -87.17869227757927 44.982806369358244, -87.16878827757004 44.93332336931216, -87.20565027760438 44.873239369256204, -87.3111232777026 44.79877336918685, -87.37873727776558 44.83774236922314, -87.4054
 1927779043 44.91120036929156, -87.3421612777315 45.01521336938843, -87.28348427767686 45.05261936942327, -87.2309152776279 45.1750633695373, -87.17791327757854 45.154973369518586, -87.06606427747438 45.29646236965036, -87.034524277445 45.290405369644716)))	
+ID	Idaho	MULTIPOLYGON (((-117.02629530537702 43.67903136814401, -117.02379430537468 43.753701368213555, -117.0371173053871 43.8001423682568, -117.02762630537825 43.83156736828607, -117.0105053053623 43.83976936829371, -117.01622030536763 43.852972368306006, -116.98577030533927 43.85935136831195, -116.97814830533218 43.8734693683251, -116.97814130533217 43.90444136835394, -116.95971630531501 43.92857736837642, -116.96795730532268 43.96319536840866, -116.93359330529069 44.01420236845617, -116.97681730533094 44.07389436851176, -116.96344330531849 44.09029836852703, -116.94688630530305 44.093025368529574, -116.9022543052615 44.1463133685792, -116.91305130527155 44.17730436860806, -116.98187130533564 44.19784236862719, -116.9761273053303 44.22518236865265, -116.99270730534573 44.24706336867303, -117.0303523053808 44.249336368675145, -117.05202730540098 44.23155636865859, -117.08138730542832 44.243846368670035, -117.10056030544618 44.26707836869167, -117.11269230545747 44.26980536869421, 
 -117.14327930548598 44.25063236867636, -117.17072330551153 44.25333236867887, -117.21357230555142 44.2847193687081, -117.21745530555505 44.30066536872295, -117.20160230554029 44.33943836875906, -117.23692130557318 44.38998236880613, -117.21722130555483 44.427855368841406, -117.22441030556152 44.47298736888344, -117.20396230554249 44.485785368895364, -117.18739130552706 44.511805368919596, -117.14516030548772 44.534655368940875, -117.14394030548658 44.55928736896381, -117.13050430547406 44.572523368976135, -117.07935430542643 44.68933636908493, -117.06651330541447 44.697557369092586, -117.03957230538938 44.749115369140604, -116.95149430530736 44.776035369165676, -116.90962030526836 44.82894036921495, -116.89736730525695 44.84855536923321, -116.86707630522874 44.86860836925189, -116.83539630519923 44.92014436929989, -116.84755630521056 44.954850369332206, -116.83139630519551 44.97263336934877, -116.84815930521111 44.97174136934794, -116.85588730521832 44.9799653693556, -116.8480973052
 1106 45.0000423693743, -116.85451330521704 45.016945369390044, -116.80730730517307 45.049755369420595, -116.78721030515436 45.075752369444814, -116.77809230514586 45.0994803694669, -116.76126830513019 45.10630036947326, -116.73658530510721 45.13730736950214, -116.68881330506271 45.26235036961859, -116.6722653050473 45.335410369686635, -116.56577230494813 45.45986336980254, -116.55450330493763 45.49364736983401, -116.4785513048669 45.56605836990144, -116.47041830485932 45.60625736993888, -116.51491530490075 45.664491369993115, -116.5282753049132 45.71072837003618, -116.56063230494334 45.74742437007035, -116.65439830503067 45.78063037010128, -116.7031803050761 45.819169370137175, -116.77370730514178 45.81976337013772, -116.79126230515813 45.84586737016203, -116.85647230521886 45.9035973702158, -116.89819730525772 45.98051637028743, -116.91913230527722 45.995175370301084, -116.95772330531315 46.06568737036676, -116.98721130534062 46.0785093703787, -116.9616373053168 46.09727437039618, 
 -116.9294263052868 46.1654833704597, -116.96749030532226 46.19755437048957, -116.97272530532712 46.24930937053777, -117.00164230535405 46.30244837058726, -117.02797330537858 46.33542737061797, -117.06418430541231 46.34869837063033, -117.04447030539394 46.38857437066747, -117.03855830538843 46.42798037070417, -117.04192630539157 46.53660137080533, -117.04096830539069 47.11931937134803, -117.04239230539201 47.25850137147765, -117.04179430539145 47.36144137157352, -117.03747230538742 47.9710923721413, -117.03886830538872 48.04618637221124, -117.02911130537964 48.838075372948744, -117.03204930538237 48.999931373099486, -116.06353130448036 48.9999503730995, -116.05669230447401 48.498665372632644, -116.0554973044729 48.208483372362394, -116.05349230447102 47.976191372146054, -116.02531630444479 47.964939372135575, -115.99893230442021 47.92514037209851, -115.93784230436331 47.86712437204447, -115.90392130433173 47.841074372020216, -115.86980930429996 47.82745237200753, -115.84932430428088 
 47.80518237198679, -115.83674230426917 47.75628137194124, -115.79053730422612 47.744838371930584, -115.77572730421232 47.70973237189789, -115.73366530417316 47.69555437188469, -115.73406730417354 47.63987937183283, -115.6982843041402 47.616080371810675, -115.69208830413444 47.590721371787055, -115.7428293041817 47.533691371733944, -115.70152230414323 47.520893371722025, -115.69277030413508 47.489540371692826, -115.64014230408605 47.4752353716795, -115.64318530408889 47.45779337166326, -115.65608730410091 47.44918037165523, -115.73248130417207 47.445303371651626, -115.75010530418848 47.43396637164106, -115.75032630418868 47.42247537163037, -115.66647730411059 47.399167371608655, -115.6387823040848 47.38004437159085, -115.59953630404824 47.3700033715815, -115.55552030400725 47.33461337154854, -115.52306430397702 47.29198237150884, -115.50193030395735 47.281644371499205, -115.42664130388722 47.274374371492435, -115.40820730387006 47.26359337148239, -115.34366130380994 47.25502237147441
 , -115.32522830379278 47.245150371465215, -115.29623430376577 47.17955037140412, -115.1930733036697 47.124026371352414, -115.17249630365053 47.09757037132778, -115.14868430362836 47.09174237132235, -115.13550730361608 47.06355037129609, -115.08133630356564 47.02652437126161, -115.0556383035417 46.97335837121209, -115.03733430352466 46.963001371202445, -115.00157430349134 46.95880937119854, -114.96473030345703 46.92521337116725, -114.92412530341922 46.90716537115044, -114.94056630343454 46.89088837113529, -114.94840930344184 46.85244637109949, -114.90232530339892 46.799433371050114, -114.86660330336565 46.797045371047886, -114.8407923033416 46.77553837102786, -114.79403030329806 46.76653137101947, -114.77783230328296 46.755717371009396, -114.78291930328771 46.70304037096034, -114.74810530325529 46.695132370952976, -114.69843130320902 46.733760370988946, -114.67388730318616 46.734721370989845, -114.6259263031415 46.6871073709455, -114.6450383031593 46.67092137093043, -114.644740303159
 01 46.66082437092102, -114.61082630312744 46.629048370891425, -114.54039130306184 46.637891370899666, -114.48471830301 46.62357437088633, -114.44153630296978 46.645715370906956, -114.3840173029162 46.66159637092174, -114.33468530287027 46.65422737091488, -114.32471230286097 46.62283937088564, -114.3433193028783 46.58788137085309, -114.35011530288463 46.51738937078744, -114.36046830289428 46.50612537077694, -114.41071530294107 46.48737137075948, -114.38402530291621 46.428179370704356, -114.3970173029283 46.399545370677686, -114.40979630294021 46.39291137067151, -114.4317953029607 46.28471137057074, -114.47379530299982 46.25296137054117, -114.47283330299892 46.243783370532626, -114.43955330296792 46.22025437051071, -114.44087930296917 46.168969370462946, -114.4670183029935 46.15526237045018, -114.50961330303318 46.15741737045219, -114.51894430304186 46.136063370432296, -114.50656830303033 46.11614237041375, -114.47737030300314 46.107357370405566, -114.45602930298327 46.082229370382166
 , -114.46575630299233 46.05081537035291, -114.49432130301894 46.02341037032738, -114.47452930300051 46.009765370314675, -114.48445530300975 45.98980637029609, -114.41244730294268 45.97197337027948, -114.40529030293601 45.95397937026272, -114.42946030295852 45.92147737023245, -114.41353030294368 45.91065137022237, -114.39283830292442 45.87088637018533, -114.4075253029381 45.84645337016258, -114.44323130297136 45.85262137016832, -114.47380330299983 45.83946837015607, -114.49916430302343 45.842683370159065, -114.51737530304041 45.81006737012869, -114.56354230308341 45.7623983700843, -114.5419583030633 45.74599937006903, -114.5349763030568 45.7229963700476, -114.49756130302194 45.69440137002097, -114.5107063030342 45.674057370002025, -114.50174130302585 45.65239336998185, -114.56467830308446 45.62427136995566, -114.54095830306237 45.5963973699297, -114.56092430308095 45.54874036988531, -114.52739230304974 45.55819336989411, -114.49659130302105 45.54664936988337, -114.46270830298948 45.5
 4784736988448, -114.43355530296233 45.527633369865654, -114.41905130294883 45.499008369839, -114.3714573029045 45.485740369826644, -114.35024630288476 45.46338336980582, -114.32643430286257 45.457424369800265, -114.26223930280278 45.485859369826755, -114.24788030278941 45.502945369842664, -114.24199830278394 45.53529036987278, -114.19480830273999 45.52791736986592, -114.17266730271938 45.54392436988083, -114.13204830268154 45.55038236988685, -114.11813930266858 45.571127369906165, -114.08296730263582 45.58637836992037, -114.05651530261119 45.62514436995647, -114.01803230257535 45.64077336997103, -114.0109903025688 45.65251136998196, -114.01987830257707 45.67237837000046, -114.00947230256737 45.68633237001346, -113.97114930253169 45.69737637002374, -113.96414430252517 45.67937837000698, -113.9266983024903 45.67121136999937, -113.92353230248735 45.65512436998439, -113.90219930246748 45.63725336996775, -113.90330530246851 45.61349136994562, -113.85202730242075 45.609562369941955, -113.
 82248530239323 45.600636369933646, -113.8037543023758 45.583729369917904, -113.83371530240369 45.514908369853806, -113.78093330235454 45.51686536985562, -113.7723043023465 45.507054369846486, -113.76916830234359 45.47770736981916, -113.78566230235894 45.44563336978929, -113.77502630234903 45.41017236975626, -113.74131030231763 45.38238636973038, -113.73908030231556 45.32153036967371, -113.68870930226865 45.27778836963297, -113.69012030226996 45.26228136961853, -113.6455923022285 45.206790369566846, -113.6009283021869 45.18099236954282, -113.59409930218054 45.14974236951372, -113.57158430215956 45.13454536949956, -113.57437630216216 45.117711369483885, -113.55227230214157 45.10754936947443, -113.51022530210243 45.107835369474685, -113.52060930211209 45.08206336945069, -113.49015930208374 45.071219369440584, -113.48630530208014 45.058321369428576, -113.4554353020514 45.04334936941463, -113.45885330205458 45.02744936939982, -113.44102930203798 44.99819436937258, -113.44876530204519 44.
 94952236932725, -113.46341330205883 44.9407753693191, -113.48734830208112 44.93957436931798, -113.49619130208936 44.930670369309695, -113.4455733020422 44.851239369235714, -113.42137930201967 44.833699369219374, -113.35002430195323 44.80756836919504, -113.34063130194447 44.779000369168436, -113.31868030192403 44.78022836916958, -113.25715430186673 44.81048636919776, -113.24033830185107 44.81184136919902, -113.13827430175601 44.761439369152086, -113.12743130174591 44.73737936912967, -113.10170330172195 44.715173369108996, -113.0989563017194 44.69591636909106, -113.07314430169536 44.67552536907207, -113.0542893016778 44.62428936902435, -113.08303730170456 44.582681368985604, -113.03966030166417 44.55629436896103, -113.03782130166246 44.532959368939295, -113.00771330163442 44.51061236891848, -113.02030930164615 44.48177636889163, -113.00665830163344 44.45261536886447, -113.01201430163843 44.43771536885059, -112.98524930161349 44.435540368848564, -112.93828130156976 44.407192368822166, 
 -112.88730730152228 44.3928523688088, -112.8707813015069 44.36997836878751, -112.84427530148221 44.35363936877229, -112.81739630145718 44.36420236878213, -112.8187103014584 44.394819368810644, -112.82669130146583 44.4210843688351, -112.79622830143747 44.45801136886949, -112.77986330142222 44.47392236888431, -112.73371230137924 44.48432036889399, -112.71432630136118 44.49693536890574, -112.65318930130424 44.48080236889072, -112.53932430119819 44.47749736888764, -112.50183930116329 44.462997368874134, -112.45851930112295 44.46883436887957, -112.42075330108777 44.44928436886136, -112.36758330103825 44.44927036886135, -112.3405773010131 44.49718036890597, -112.3425073010149 44.52510036893197, -112.28234130095886 44.54170236894744, -112.25667530093496 44.55997236896445, -112.2303983009105 44.559491368964004, -112.21776330089872 44.53849536894445, -112.19965830088186 44.531449368937885, -112.12419030081158 44.52825336893491, -112.09989730078895 44.518231368925576, -112.0593673007512 44.52
 8611368935245, -112.02707730072113 44.52284336892987, -112.02361330071791 44.53504336894123, -111.97781830067525 44.52967636893624, -111.9403863006404 44.54972636895491, -111.87250230057717 44.556265368961, -111.80783730051695 44.503982368912304, -111.79260830050276 44.518462368925796, -111.76691830047884 44.51882536892613, -111.71699730043235 44.53376036894004, -111.68486230040241 44.55075236895587, -111.60524830032827 44.54298936894864, -111.56723130029286 44.55286636895784, -111.49024130022117 44.528697368935326, -111.48257330021403 44.53614336894226, -111.45932530019238 44.53792136894391, -111.46282730019563 44.549942368955115, -111.49290430022364 44.55118936895627, -111.51452630024379 44.59319736899539, -111.50174730023188 44.615971369016606, -111.50769030023741 44.63768836903683, -111.47016830020247 44.640710369039645, -111.45826530019139 44.652555369050674, -111.46069230019364 44.67002336906694, -111.48080430021237 44.691416369086866, -111.47542530020736 44.702162369096875, -
 111.44363230017775 44.71317936910714, -111.39508430013254 44.70886936910313, -111.38495930012311 44.73769436912997, -111.37230930011133 44.745087369136854, -111.34997730009053 44.72617736911924, -111.31922130006188 44.72786436912081, -111.3154753000584 44.7051933690997, -111.29566830003995 44.68293836907897, -111.27020830001624 44.673802369070465, -111.27066530001667 44.64221236904105, -111.22397129997319 44.62690836902679, -111.2197972999693 44.61798136901848, -111.23423329998273 44.60256236900412, -111.21950729996902 44.57317036897675, -111.17876429993107 44.564851368968995, -111.17024229992315 44.54518636895068, -111.13435929988972 44.52790236893458, -111.12891829988466 44.500757368909305, -111.09463029985272 44.48612436889567, -111.0515602998126 44.473323368883754, -111.05040529981153 43.98255336842669, -111.04677129980816 43.515528367991735, -111.04749829980882 43.28473436777679, -111.04921529981043 43.01988336753013, -111.04678029980816 42.50325236704899, -111.04869729980994 4
 1.99620336657675, -111.49458630022521 42.000171366580446, -112.10051430078953 42.00230036658243, -112.14711630083293 41.999054366579415, -112.98957530161753 42.00114636658136, -114.03907230259495 41.995391366576, -114.26947130280954 41.995924366576496, -115.02486330351303 41.99650636657704, -115.94754430437234 41.99459936657526, -116.99231330534536 41.99479436657545, -117.0188643053701 41.99479436657545, -117.02629530537702 43.67903136814401)))	
+VT	Vermont	MULTIPOLYGON (((-73.25806026461467 42.74605836727511, -73.26927526462511 42.747481367276436, -73.29616926465016 42.80354936732866, -73.27958326463471 42.8371033673599, -73.27600526463138 42.940294367456005, -73.25007126460723 43.31085436780112, -73.23839126459634 43.512832367989226, -73.25998426461646 43.55938236803258, -73.29140226464573 43.57503336804716, -73.28173626463672 43.593187368064065, -73.29410426464824 43.619653368088706, -73.30353426465702 43.62471436809342, -73.36368526471304 43.61499836808437, -73.38811426473579 43.569143368041665, -73.41832026476392 43.582479368054095, -73.42296026476825 43.63211436810032, -73.37098926471984 43.71428136817684, -73.35666926470651 43.75655836821622, -73.35899726470868 43.77842736823658, -73.38474026473266 43.80450836826087, -73.37512126472369 43.88597736833674, -73.40533426475183 43.9148073683636, -73.41740626476307 43.98819736843194, -73.40825126475454 44.0182223684599, -73.43600026478039 44.04567936848548, -73.435215264779
 66 44.063897368502445, -73.40875726475501 44.10661036854222, -73.4078652647542 44.136227368569806, -73.38206226473015 44.17210736860322, -73.37733226472575 44.20124736863036, -73.30532526465869 44.26014236868521, -73.32978826468147 44.367390368785095, -73.29999526465373 44.40553336882061, -73.29331926464751 44.43285336884606, -73.33445226468581 44.54432836894988, -73.34781226469826 44.55397136895886, -73.37129626472013 44.579167368982326, -73.38182526472994 44.61980736902018, -73.37013626471905 44.63434936903372, -73.3730972647218 44.6612763690588, -73.35815126470789 44.680368369076575, -73.37315826472187 44.724236369117435, -73.32678626467867 44.79929336918734, -73.36905426471805 44.819118369205796, -73.38230626473039 44.847933369232635, -73.33641426468765 44.93260436931149, -73.350758264701 44.98197336935747, -73.34472326469538 45.006138369379975, -73.18854626454993 45.00848636938216, -72.54723126395265 45.005370369379264, -71.90186826335162 45.0073403693811, -71.50537226298235 45
 .0133513693867, -71.54092726301546 44.976563369352434, -71.51697726299317 44.94369636932182, -71.50636526298328 44.899671369280824, -71.57510126304729 44.81601936920291, -71.58350126305511 44.77919736916862, -71.63113326309947 44.741710369133706, -71.60767826307763 44.67786236907425, -71.58874926306 44.650599369048855, -71.5680272630407 44.6374463690366, -71.55410226302773 44.59658936899855, -71.53679126301161 44.57893136898211, -71.5922882630633 44.55120336895628, -71.5914412630625 44.5388743689448, -71.57524326304743 44.52580536893263, -71.58661926305801 44.49453736890351, -71.61422326308373 44.47450736888486, -71.63655426310453 44.47673136888693, -71.64770926311492 44.46917436887989, -71.656399263123 44.440137368852845, -71.67688426314209 44.42134236883534, -71.76657026322562 44.39824836881384, -71.79772926325462 44.384172368800726, -71.82119726327649 44.35036036876923, -71.83481626328917 44.3441993687635, -71.92836126337629 44.33611236875596, -71.9389052633861 44.32578636874635,
  -71.99443326343783 44.32754836874799, -72.03549526347607 44.299434368721805, -72.05956626349848 44.26149436868647, -72.04439026348435 44.23437936866122, -72.05928226349822 44.1821763686126, -72.04472426348467 44.15643536858863, -72.03492026347553 44.12074636855539, -72.04951526348913 44.100452368536494, -72.03244726347323 44.096099368532435, -72.03472826347536 44.08337436852058, -72.07691926351464 44.03204036847278, -72.08520426352236 44.00892436845125, -72.10990926354538 43.9892293684329, -72.11280826354808 43.97651536842106, -72.09171126352842 43.95799136840381, -72.11320426354844 43.93916636838628, -72.12164926355631 43.909217368358384, -72.17008926360141 43.878917368330164, -72.18483626361515 43.80169036825825, -72.20609226363494 43.764635368223736, -72.21912326364708 43.75069236821075, -72.2600552636852 43.73530036819642, -72.30404026372616 43.69853036816217, -72.33308526375322 43.59736436806796, -72.37349826379085 43.57237436804468, -72.39499826381088 43.517554367993625, -72.
 38251526379925 43.48462936796296, -72.39624826381204 43.410156367893606, -72.41213926382684 43.37712536786284, -72.39762826381333 43.351006367838515, -72.41023126382507 43.323404367812806, -72.40241926381779 43.307382367797885, -72.43559826384869 43.23225336772792, -72.45239826386434 43.15602236765692, -72.43760526385056 43.1162703676199, -72.44346426385601 43.079039367585224, -72.46175226387305 43.046504367554924, -72.45715926386877 42.99960336751124, -72.47334126388384 42.9761433674894, -72.50426326391263 42.965584367479565, -72.5202172639275 42.9516723674666, -72.52481026393178 42.91261436743023, -72.55342826395842 42.860643367381826, -72.53891726394491 42.80773336733255, -72.51306826392084 42.789259367315346, -72.50726926391545 42.768732367296224, -72.47932226388942 42.761588367289576, -72.46217126387344 42.74684036727584, -72.45577026386748 42.725852367256294, -72.92299726430262 42.73736436726702, -73.01969526439268 42.74039636726984, -73.25806026461467 42.74605836727511)))	
+MN	Minnesota	MULTIPOLYGON (((-91.73036628181835 43.49957136797688, -92.07753228214168 43.49915336797649, -92.45316928249152 43.499462367976776, -92.55800828258914 43.50025936797752, -93.02721128302613 43.501278367978465, -93.05438028305143 43.50145736797863, -93.50083028346722 43.50048836797773, -93.65369928360958 43.500762367977984, -93.97395028390784 43.50029836797755, -94.24678728416194 43.4989483679763, -94.45523828435608 43.498102367975505, -94.8598392847329 43.5000303679773, -94.92046428478936 43.49937136797669, -95.39655828523276 43.50033436797759, -95.46477528529628 43.499541367976846, -95.86691228567081 43.49894436797629, -96.0610392858516 43.49853336797591, -96.46045428622358 43.49971836797701, -96.45660228621999 43.848741368302065, -96.45739728622074 44.19906136862833, -96.45510628621861 44.53834336894431, -96.45671828622011 44.62880836902856, -96.45521728621871 44.801347369189244, -96.4560802862195 44.97199436934818, -96.45449628621803 45.27519536963055, -96.457602286220
 93 45.298850369652584, -96.47759228623954 45.328509369680205, -96.53254928629073 45.37513236972363, -96.60508428635828 45.39652436974355, -96.69316928644032 45.4106383697567, -96.7380322864821 45.45819536980099, -96.76924628651116 45.5174783698562, -96.84308728657994 45.584090369918236, -96.85499028659102 45.609122369941545, -96.83279628657036 45.65068736998026, -96.65739128640699 45.738970370062475, -96.60461028635784 45.80826437012701, -96.58795528634232 45.81785437013595, -96.56692128632274 45.93411037024421, -96.56180228631797 45.947683370256854, -96.57621528633139 46.0212793703254, -96.55193128630877 46.09552937039455, -96.57116628632669 46.17717437047059, -96.58789028634227 46.191918370484316, -96.58645628634093 46.2154133705062, -96.59818328635185 46.23868237052787, -96.60207428635547 46.33632437061881, -96.61486128636739 46.3508123706323, -96.65210128640207 46.35943337064033, -96.68822828643572 46.41221837068949, -96.70968228645569 46.42716837070341, -96.71489428646055 46.46
 871837074211, -96.74031628648423 46.4894323707614, -96.75122728649438 46.58861937085378, -96.77104128651284 46.59998337086436, -96.78431728652521 46.624112370886834, -96.79024628653073 46.6297733708921, -96.79369528653393 46.67880437093777, -96.78155628652263 46.70704437096407, -96.78038228652154 46.76231237101554, -96.7971972865372 46.812033371061844, -96.76825028651024 46.84486137109242, -96.77806128651937 46.86734937111336, -96.75691128649969 46.92278037116499, -96.76306828650542 46.936261371177544, -96.78792528652856 46.93218437117375, -96.78971028653022 46.948202371188664, -96.80188728654157 46.95584337119578, -96.79342528653369 46.96964137120863, -96.81677228655543 46.96977937120876, -96.82453128656266 47.003436371240106, -96.83529628657269 47.010231371246434, -96.82260828656086 47.033932371268506, -96.82696428656492 47.07883237131033, -96.81915128655764 47.09260437132315, -96.83916428657629 47.15188637137836, -96.82649128656448 47.17006337139529, -96.83706528657433 47.2404583
 71460846, -96.84962328658602 47.25684337147611, -96.83771428657494 47.2938843715106, -96.84674728658335 47.3146023715299, -96.83846128657564 47.34224337155564, -96.85063128658696 47.36095437157307, -96.8398272865769 47.38411737159464, -96.85000528658638 47.408936371617756, -96.86724828660245 47.41308737162162, -96.8558272865918 47.43675337164366, -96.86668428660191 47.46153737166674, -96.85161528658789 47.50061937170314, -96.86068728659633 47.521356371722455, -96.84918828658562 47.54456837174408, -96.85866428659445 47.56297837176122, -96.85221728658844 47.601151371796774, -96.87333528660811 47.61525537180991, -96.8894252866231 47.67392537186454, -96.92365928665498 47.71409437190196, -96.93201228666275 47.763506371947976, -96.9578302866868 47.79444037197678, -96.98389328671108 47.809661371990956, -96.97723128670486 47.82802937200807, -97.0003402867264 47.87019737204734, -97.02056628674522 47.87556937205234, -97.01533128674035 47.917890372091755, -97.04805328677082 47.95492437212624, 
 -97.06707128678855 48.04816437221308, -97.09272128681243 48.07034437223374, -97.0990302868183 48.10097237226226, -97.12187328683957 48.11636937227661, -97.12091828683869 48.1427743723012, -97.13651328685322 48.14839837230643, -97.11606528683417 48.15922337231652, -97.13744328685408 48.16776937232447, -97.13629128685301 48.17522737233142, -97.13727528685392 48.19506337234989, -97.13082828684792 48.20374237235798, -97.11089928682935 48.20760537236157, -97.13975428685623 48.22175537237475, -97.10923528682781 48.22804937238061, -97.12755428684487 48.23352337238571, -97.12378428684136 48.259173372409606, -97.13665528685334 48.264483372414546, -97.11171428683012 48.277876372427016, -97.11268328683101 48.286147372434726, -97.13051328684763 48.29304037244114, -97.11372128683199 48.294882372442856, -97.11475128683294 48.303618372451, -97.1326342868496 48.31096937245784, -97.11259128683093 48.319926372466185, -97.13713628685379 48.32599137247183, -97.1311232868482 48.361491372504894, -97.1503
 9628686614 48.3632153725065, -97.13378628685068 48.3724543725151, -97.135205286852 48.38441037252623, -97.15881928687399 48.38820637252977, -97.12912428684633 48.4078853725481, -97.14982328686561 48.40999137255006, -97.1516472868673 48.41961237255902, -97.12260128684025 48.416110372555764, -97.1196332868375 48.43710237257531, -97.14361328685983 48.43810937257625, -97.13459428685142 48.51731437265001, -97.14832728686422 48.51795137265061, -97.13938528685588 48.534648372666155, -97.15553728687092 48.53839837266965, -97.1604352868755 48.545078372675874, -97.14661828686262 48.54953737268002, -97.16794328688249 48.56226337269187, -97.15212728686775 48.572856372701736, -97.1581922868734 48.583640372711784, -97.14081228685721 48.586905372714824, -97.14471828686085 48.61402437274008, -97.12295828684059 48.62076837274636, -97.12744428684476 48.62979437275477, -97.1076302868263 48.629946372754915, -97.09716928681657 48.67452937279643, -97.1167392868348 48.695243372815725, -97.11010128682861 4
 8.708583372828144, -97.13480628685163 48.72623837284459, -97.13250228684947 48.747218372864126, -97.14789828686382 48.75565337287198, -97.13924628685575 48.76354237287933, -97.14751628686346 48.781170372895744, -97.17394428688807 48.801514372914696, -97.16471228687948 48.81036837292294, -97.1804222868941 48.81553737292775, -97.17120428688553 48.83598037294679, -97.17572728688974 48.87375737298198, -97.21636928692759 48.931830373036064, -97.22943628693976 48.999987373099536, -96.40691528617373 48.999982373099535, -95.27665728512109 48.99999137309954, -95.15775028501035 48.99999637309955, -95.15186728500487 49.37173037344575, -94.83203928470701 49.33080637340764, -94.68125028456657 48.87716137298514, -94.69443228457885 48.77761537289244, -94.57031228446326 48.71367637283289, -94.43063428433317 48.7107853728302, -94.29233728420436 48.70771137282733, -94.23082728414708 48.65198737277544, -93.84390428378673 48.62473737275006, -93.81268528375766 48.52540837265755, -93.78110628372825 48.51
 159037264468, -93.51413928347961 48.534271372665806, -93.46533928343416 48.54952037268001, -93.45776928342711 48.592710372720234, -93.30423628328413 48.637163372761634, -93.09144228308595 48.62658437275178, -92.94692628295135 48.62835537275343, -92.7290002827484 48.54021137267134, -92.6418202826672 48.54034937267147, -92.62638028265282 48.50282437263652, -92.69882128272029 48.49472137262897, -92.70664328272757 48.460370372596984, -92.49752928253282 48.44007237257807, -92.45634528249447 48.40216937254277, -92.47332228251028 48.357499372501174, -92.37011628241416 48.22077937237384, -92.27691828232736 48.24434037239578, -92.30027228234911 48.29831137244605, -92.27613128232663 48.35231937249635, -92.12596228218678 48.3667563725098, -92.03518328210222 48.355508372499315, -91.9795342820504 48.25039837240143, -91.78881528187279 48.20614537236021, -91.71193828180118 48.19677537235149, -91.70373128179354 48.11483537227518, -91.56877528166785 48.104457372265514, -91.57156228167045 48.04357137
 22088, -91.23944628136114 48.08129837224394, -91.02714828116342 48.19533937235015, -90.86449528101194 48.25419837240497, -90.74336528089913 48.088443372250595, -90.5674552807353 48.12169937228157, -90.55683528072541 48.092750372254606, -90.1452702803421 48.11277037227325, -90.02670028023168 48.08607937224839, -89.98702028019473 48.023556372190164, -89.90038928011404 47.99250537216125, -89.74931027997334 48.0264843721929, -89.53067327976972 48.00165637216977, -89.62564527985818 47.9925613721613, -89.63637327986817 47.95939037213041, -89.99967728020651 47.82456437200484, -90.50963328068146 47.70993837189809, -91.02147528115815 47.4610583716663, -91.46865728157461 47.12493537135326, -91.8009692818841 46.927086371169, -92.08849228215188 46.79189737104309, -92.21462428226934 46.668204370927896, -92.30314828235178 46.66657537092638, -92.287271282337 46.658786370919124, -92.28868528233832 46.415984370692996, -92.28894428233856 46.15660037045143, -92.28937028233896 46.07323137037378, -92.32
 737228237436 46.056878370358554, -92.34622528239191 46.022596370326625, -92.36496328240936 46.01624837032071, -92.42499928246528 46.02550437032933, -92.46234528250005 45.98119737028807, -92.52397728255745 45.98258337028936, -92.55267228258418 45.9512693702602, -92.66620828268992 45.91570337022708, -92.7062402827272 45.89095837020403, -92.73409728275314 45.84498037016121, -92.7487622827668 45.837302370154056, -92.77910728279507 45.763340370085174, -92.83363628284584 45.73089037005495, -92.86001928287041 45.71056237003602, -92.88539728289405 45.64495536997492, -92.87683128288607 45.57883636991335, -92.83503728284715 45.56340236989897, -92.7621752827793 45.56426336989977, -92.72815428274761 45.54724236988392, -92.68542128270781 45.470053369812035, -92.6548172826793 45.45522136979822, -92.64497528267015 45.43945236978353, -92.64875128267366 45.395466369742564, -92.6848692827073 45.3630763697124, -92.70738428272827 45.318201369670604, -92.74659328276478 45.297603369651426, -92.7554192827
 73 45.21237636957205, -92.76258328277967 45.18661236954806, -92.74493528276324 45.15642236951994, -92.74542228276368 45.1130043694795, -92.7967622828115 45.06561036943536, -92.76299128278005 45.02211936939486, -92.7671262827839 45.00100536937519, -92.74976828276773 44.93565536931433, -92.75392628277162 44.915002369295095, -92.77187128278833 44.899496369280655, -92.76426328278124 44.862234369245954, -92.76102828277823 44.83537136922094, -92.80558428281972 44.746160369137854, -92.73714528275598 44.713594369107525, -92.63036728265654 44.64265236904146, -92.60897328263661 44.61029236901132, -92.5092152825437 44.575159368978596, -92.34087228238693 44.5528353689578, -92.32047828236793 44.540491368946306, -92.29668728234577 44.49218236890132, -92.24910028230146 44.45621636886782, -92.20613728226144 44.43839436885122, -92.09133328215452 44.415589368829984, -91.97238628204374 44.36448736878239, -91.93886828201254 44.33911136875876, -91.92275428199753 44.31752036873865, -91.92234928199714 44.
 28834136871147, -91.8886942819658 44.25749536868275, -91.84874428192859 44.191187368620994, -91.75321928183963 44.137227368570734, -91.65223328174558 44.066895368505236, -91.6017862816986 44.04082236848096, -91.56916228166821 44.034955368475494, -91.52842028163028 44.034215368474804, -91.42590228153479 43.98561936842954, -91.37335728148585 43.94719136839375, -91.29194828141004 43.847190368300616, -91.251105281372 43.788075368245565, -91.25891628137927 43.722395368184394, -91.25838928137878 43.67732236814242, -91.23299028135513 43.59889036806938, -91.24055828136218 43.54871236802264, -91.22356628134635 43.500808367978024, -91.61109928170727 43.50062636797786, -91.73036628181835 43.49957136797688)))	
+OR	Oregon	MULTIPOLYGON (((-121.441509309489 41.99433436657502, -122.28470531027429 42.000764366581, -123.22210231114731 42.00219136658233, -123.51320431141842 41.99783336657828, -123.81914631170335 41.99294836657373, -124.20644431206405 41.997648366578105, -124.35224631219984 42.09867736667219, -124.41506231225834 42.245894366809296, -124.43781831227955 42.429608366980396, -124.39176331223663 42.55302736709534, -124.40107831224532 42.62269936716022, -124.55961731239297 42.83245736735557, -124.4853463123238 42.955454367470125, -124.386772312232 43.261589367755235, -124.40607631224998 43.3001973677912, -124.27399431212697 43.45910536793919, -124.22600431208227 43.60500436807507, -124.15832531201923 43.85711836830987, -124.11831931198198 44.269515368693945, -124.05440531192245 44.6621393690596, -124.07556831194216 44.81473836920172, -124.00757231187885 45.03610336940788, -123.95660731183138 45.292965369647106, -123.98056031185368 45.485084369826026, -123.93667431181281 45.5079663698473
 4, -123.8921083117713 45.47405036981576, -123.85950731174094 45.499082369839066, -123.95341531182

<TRUNCATED>

[12/26] lucenenet git commit: Added missing Spatial classes, ported Spatial tests, and fixed several bugs

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Test-Files/Data/geonames-IE.txt
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Test-Files/Data/geonames-IE.txt b/src/Lucene.Net.Tests.Spatial/Test-Files/Data/geonames-IE.txt
new file mode 100644
index 0000000..baa7b74
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Test-Files/Data/geonames-IE.txt
@@ -0,0 +1,22929 @@
+2652852	Cliff	Cliff		54.48333	-8.11667	S	EST	IE		U	06			0		76	Europe/Dublin	2010-08-14
+2654332	Bun Cranncha	Bun Cranncha	Buncrana	55.13333	-7.45	P	PPL	IE		U	06			5546		63	Europe/Dublin	2010-08-14
+2960863	Dromore West	Dromore West	Dromore,Dromore West	54.25167	-8.89306	P	PPL	IE	IE	C	25			0		32	Europe/Dublin	2010-08-14
+2960864	Dromore	Dromore	Dromore,Dromore House	52.93111	-8.96833	S	EST	IE	IE	M	03			0		36	Europe/Dublin	2010-08-14
+2960865	Youghal Harbour	Youghal Harbour		51.95194	-7.83972	H	HBR	IE		00	00			0		1	Europe/Dublin	2010-08-14
+2960866	Youghal Bay	Youghal Bay		52.89944	-8.30361	H	BAY	IE		M	26			0		55	Europe/Dublin	2010-08-14
+2960867	Youghal Bay	Youghal Bay		51.91	-7.81917	H	BAY	IE		00	00			0		-9999	Europe/Dublin	2010-08-14
+2960868	Youghal	Youghal		52.88861	-8.3075	P	PPL	IE		M	26			0		61	Europe/Dublin	2010-08-14
+2960869	Eochaill	Eochaill	Youghal	51.95	-7.85056	P	PPL	IE		M	04			6868		1	Europe/Dublin	2010-08-14
+2960870	Yellow River	Yellow River		54.13972	-8.02722	H	STM	IE		C	14			0		58	Europe/Dublin	2010-08-14
+2960871	Yellow River	Yellow River		54.05	-7.83333	H	STM	IE		C	14			0		76	Europe/Dublin	2010-08-14
+2960872	Yellow River	Yellow River		53.81667	-9	H	STM	IE		C	20			0		51	Europe/Dublin	2010-08-14
+2960873	Yellow River	Yellow River		53.38556	-7.08361	H	STM	IE		00	00			0		108	Europe/Dublin	2010-08-14
+2960874	Yellow Furze	Yellow Furze		53.67333	-6.57167	P	PPL	IE		L	21			0		86	Europe/Dublin	2010-08-14
+2960875	Woodville	Woodville		54.13333	-9.31667	P	PPL	IE		C	20			0		42	Europe/Dublin	2010-08-14
+2960876	Woodville	Woodville		53.85	-7.56667	S	BLDG	IE		L	18			0		71	Europe/Dublin	2010-08-14
+2960877	Wood View	Wood View		52.83333	-7.21667	S	EST	IE		L	13			0		135	Europe/Dublin	2010-08-14
+2960878	Woodtown House	Woodtown House	Woodtown,Woodtown House	53.58139	-6.98639	S	EST	IE	IE	L	21			0		92	Europe/Dublin	2010-08-14
+2960879	Woodstown	Woodstown		52.19667	-7.01	P	PPL	IE		M	27			0		75	Europe/Dublin	2010-08-14
+2960880	Woodstock House	Woodstock House	Woodstock,Woodstock House	52.47861	-7.05639	S	EST	IE	IE	L	13			0		32	Europe/Dublin	2010-08-14
+2960881	Woodsgift House	Woodsgift House	Woodsgift,Woodsgift House	52.70694	-7.515	S	EST	IE	IE	L	13			0		152	Europe/Dublin	2010-08-14
+2960882	Woodrooff House	Woodrooff House	Woodroof House,Woodrooff House	52.38222	-7.81944	S	EST	IE	IE	M	26			0		122	Europe/Dublin	2010-11-04
+2960883	Woodpark	Woodpark		53.44417	-6.48417	S	EST	IE		L	21			0		78	Europe/Dublin	2010-08-14
+2960884	Woodpark	Woodpark		52.92583	-8.45833	S	EST	IE		M	03			0		53	Europe/Dublin	2010-08-14
+2960885	Woodmount	Woodmount		53.33583	-8.16167	S	EST	IE		C	24			0		73	Europe/Dublin	2010-08-14
+2960886	Wood Lodge	Wood Lodge		54.03333	-7.16667	S	BLDG	IE		U	02			0		111	Europe/Dublin	2010-08-14
+2960887	Woodlawn Station	Woodlawn Station	Woodlawn,Woodlawn Station	53.35	-8.46667	S	RSTN	IE	IE	C	10			0		76	Europe/Dublin	2010-08-14
+2960888	Woodlawn	Woodlawn	Woodlawn,Woodlawn House	53.32889	-8.47889	S	EST	IE	IE	C	10			0		76	Europe/Dublin	2010-08-14
+2960889	Woodlands Station	Woodlands Station	Woodlands,Woodlands Station	51.55	-9.51667	S	RSTN	IE	IE	M	04			0		145	Europe/Dublin	2010-08-14
+2960890	Woodhouse	Woodhouse		52.12389	-7.475	S	EST	IE		M	27			0		1	Europe/Dublin	2010-08-14
+2960891	Wood Hill	Wood Hill		54.75944	-8.41833	S	EST	IE		U	06			0		32	Europe/Dublin	2010-08-14
+2960892	Woodfort	Woodfort	Woodfoot,Woodfort	53.88333	-6.83333	P	PPL	IE		L	21			0		111	Europe/Dublin	2010-08-10
+2960893	Woodfort	Woodfort		52.15944	-8.34139	S	EST	IE		M	04			0		40	Europe/Dublin	2010-08-14
+2960894	Woodford River	Woodford River	Rosmore,Woodford River	53.02028	-8.31639	H	STM	IE	IE	C	10			0		55	Europe/Dublin	2010-08-14
+2960895	Woodford	Woodford	An Chraig,An Chr�ig,An Ghraig,An Ghr�ig,Woodford	53.04833	-8.40028	P	PPL	IE		C	10			0		76	Europe/Dublin	2010-08-14
+2960896	Woodford	Woodford		52.05	-9.45	P	PPL	IE		M	11			0		92	Europe/Dublin	1993-12-27
+2960897	Woodfield House	Woodfield House		53.86667	-8.88333	S	EST	IE		C	20			0		70	Europe/Dublin	1993-12-27
+2960898	Woodfield House	Woodfield House		53.37028	-7.62167	S	EST	IE		L	23			0		74	Europe/Dublin	2010-08-14
+2960899	Woodfield House	Woodfield House		53.11667	-7.9	S	EST	IE		L	23			0		70	Europe/Dublin	2010-08-14
+2960900	Woodenbridge Junction Station	Woodenbridge Junction Station	Woodenbridge,Woodenbridge Junction,Woodenbridge Junction Station	52.83333	-6.23333	S	RSTN	IE	IE	L	31			0		40	Europe/Dublin	2010-08-14
+2960901	Woodenbridge	Woodenbridge		52.83222	-6.23639	P	PPL	IE		L	31			0		43	Europe/Dublin	2010-08-14
+2960902	Woodbrook House	Woodbrook House	Woodbrook,Woodbrook House	53.12944	-7.23583	S	HSEC	IE	IE	L	15			0		101	Europe/Dublin	2010-08-14
+2960903	Woodbrook	Woodbrook		52.54806	-6.73861	S	EST	IE		L	30			0		131	Europe/Dublin	2010-08-14
+2960904	Woodbrook	Woodbrook		51.71667	-9.13333	S	EST	IE		M	04			0		89	Europe/Dublin	1993-12-27
+2960905	Woodbine Hill	Woodbine Hill		51.95472	-7.82944	S	EST	IE		M	04			0		1	Europe/Dublin	2010-08-14
+2960906	Wingfield House	Wingfield House		52.7725	-6.39194	S	EST	IE		L	30			0		154	Europe/Dublin	2010-08-14
+2960907	Windy Harbour	Windy Harbour		53.74083	-6.66472	P	PPL	IE		L	21			0		130	Europe/Dublin	2010-08-14
+2960908	Windy Gap	Windy Gap		52.03333	-9.93333	T	GAP	IE		M	11			0		269	Europe/Dublin	2010-08-14
+2960909	Windgap	Windgap	Bearna na Gaoithe,Bearra na Gaoithe,Windgap	52.46556	-7.39972	P	PPL	IE		L	13			0		152	Europe/Dublin	2010-08-14
+2960910	Windfield House	Windfield House		53.43333	-8.63333	S	EST	IE		C	10			0		76	Europe/Dublin	1993-12-27
+2960911	Wilton House	Wilton House	Wilton,Wilton House	52.72222	-7.54806	S	EST	IE	IE	L	13			0		137	Europe/Dublin	2010-08-14
+2960912	Wilton Castle	Wilton Castle	Wilton Castle,Wilton House	52.45917	-6.61639	S	EST	IE	IE	L	30			0		63	Europe/Dublin	2010-08-14
+2960913	Wilmount House	Wilmount House		52.58333	-6.71667	S	EST	IE		L	30			0		150	Europe/Dublin	2010-08-14
+2960914	Wilmount House	Wilmount House		52.41667	-6.45611	S	EST	IE		L	30			0		24	Europe/Dublin	2010-08-14
+2960915	Wilmount	Wilmount		53.74833	-6.89528	S	EST	IE		L	21			0		71	Europe/Dublin	2010-08-14
+2960916	Wills Grove	Wills Grove		53.73333	-8.41667	S	RUIN	IE		C	24			0		76	Europe/Dublin	1993-12-27
+2960917	Willsbrook House	Willsbrook House		53.73333	-8.48333	S	BLDG	IE		C	24			0		74	Europe/Dublin	1993-12-27
+2960918	Willmount House	Willmount House		53.11611	-6.55833	S	HSEC	IE		L	31			0		198	Europe/Dublin	2010-08-14
+2960919	Willmount	Willmount		53.26667	-8.76667	S	EST	IE		C	10			0		30	Europe/Dublin	1993-12-27
+2960920	Willmount	Willmount		52.55472	-7.56556	S	EST	IE		M	26			0		150	Europe/Dublin	2010-08-14
+2960921	Williamstown Lodge	Williamstown Lodge		52.95	-8.33333	S	BLDG	IE		M	03			0		55	Europe/Dublin	2010-08-14
+2960922	Williamstown House	Williamstown House		53.88333	-6.41417	S	EST	IE		L	19			0		3	Europe/Dublin	2010-08-14
+2960923	Williamstown Harbour	Williamstown Harbour		52.95	-8.33333	H	HBR	IE		M	03			0		55	Europe/Dublin	2010-08-14
+2960924	Williamstown	Williamstown		53.68333	-8.58333	P	PPL	IE		C	10			0		76	Europe/Dublin	1993-12-27
+2960925	Williamstown	Williamstown		53.75222	-6.87333	S	EST	IE		L	21			0		70	Europe/Dublin	2010-08-14
+2960926	Willbrook	Willbrook		53.29361	-6.31194	P	PPL	IE		L	07			0		76	Europe/Dublin	2010-08-14
+2960927	Willbrook	Willbrook		52.93778	-9.12806	P	PPL	IE		M	03			0		95	Europe/Dublin	2010-08-14
+2960928	Wilkinstown	Wilkinstown		53.73556	-6.71306	P	PPL	IE		L	21			0		73	Europe/Dublin	2010-08-14
+2960929	Wilford House	Wilford House		52.55722	-7.55889	S	EST	IE		M	26			0		150	Europe/Dublin	2010-08-14
+2960930	Wild Bellows Rock	Wild Bellows Rock		53.34417	-10.05056	T	RK	IE		C	10			0		-9999	Europe/Dublin	1999-03-04
+2960931	Wicklow Mountains	Wicklow Mountains	Wicklow Hills,Wicklow Mountains	53.08333	-6.33333	T	MTS	IE		L	31			0		439	Europe/Dublin	2010-08-10
+2960932	Wicklow Head	Wicklow Head		52.96056	-5.99889	T	CAPE	IE		L	31			0		-9999	Europe/Dublin	2010-08-14
+2960933	Wicklow Gap	Wicklow Gap		53.04056	-6.40194	T	GAP	IE		L	31			0		528	Europe/Dublin	2010-08-14
+2960934	Wicklow Gap	Wicklow Gap		52.75667	-6.35944	T	GAP	IE		L	30			0		160	Europe/Dublin	2010-08-14
+2960935	Cill Mhant�in	Cill Mhantain	Contae Chill Mhantain,Contae Chill Mhant�in,County Wicklow,Wicklow	53	-6.41667	A	ADM2	IE		L	31			119873		371	Europe/Dublin	2010-08-14
+2960936	Cill Mhant�in	Cill Mhantain	Cill Maintain,Cill Maintainn,Cill Maint�inn,Cill Manntein,Cill Mannt�in,Cill Mantan,Cill Mhanntain,Wicklow	52.975	-6.04944	P	PPLA2	IE		L	31			10461		69	Europe/Dublin	2010-08-14
+2960937	Whiting Bay	Whiting Bay		51.94694	-7.78139	H	BAY	IE		M	04			0		-9999	Europe/Dublin	2010-08-14
+2960938	Whitfield Court	Whitfield Court	Whitfield,Whitfield Court	52.22278	-7.21722	S	EST	IE	IE	M	27			0		54	Europe/Dublin	2010-08-14
+2960939	Whitewood House	Whitewood House	Whitewood,Whitewood House	53.84139	-6.78333	S	EST	IE	IE	L	21			0		72	Europe/Dublin	2010-08-14
+2960940	Whitewell House	Whitewell House		53.43333	-7.31667	S	EST	IE		L	29			0		130	Europe/Dublin	1993-12-27
+2960941	Whites Town	Whites Town		53.99778	-6.12778	P	PPL	IE		L	19			0		1	Europe/Dublin	2010-08-14
+2960942	White River	White River		53.85972	-6.38694	H	STM	IE		L	19			0		8	Europe/Dublin	2010-08-14
+2960943	White River	White River		52.59556	-9.19361	H	STM	IE		M	16			0		15	Europe/Dublin	2010-08-14
+2960944	White Point	White Point		51.85	-8.31667	T	PT	IE		M	04			0		-9999	Europe/Dublin	2010-08-14
+2960945	White Mountain	White Mountain		52.48333	-6.83333	T	MT	IE		L	30			0		215	Europe/Dublin	2010-08-14
+2960946	White Lough	White Lough		53.70194	-7.22	H	LK	IE		00				0		151	Europe/Dublin	1999-03-22
+2960947	Whitehill House	Whitehill House		53.73333	-7.63333	S	EST	IE		L	18			0		76	Europe/Dublin	1993-12-27
+2960948	Whitehall	Whitehall		53.75	-7.93333	P	PPL	IE		C	24			0		66	Europe/Dublin	1993-12-27
+2960949	Paulstown	Paulstown	Baile Phoil,Baile Ph�il,Paulstown,Whitehall	52.68111	-7.02222	P	PPL	IE		L	13			0		55	Europe/Dublin	2010-08-14
+2960950	Whitegate	Whitegate		53.73333	-7.31667	P	PPL	IE		L	29			0		211	Europe/Dublin	1993-12-27
+2960951	Whitegate	Whitegate	An Geata Ban,An Geata B�n,Whitegate	52.94917	-8.37139	P	PPL	IE		M	03			0		56	Europe/Dublin	2010-08-14
+2960952	Whitegate	Whitegate	An Geata Ban,An Geata B�n,Whitegate	51.83056	-8.22972	P	PPL	IE		M	04			0		1	Europe/Dublin	2010-08-14
+2960953	Whitechurch	Whitechurch		52.37139	-7.39306	P	PPL	IE		L	13			0		32	Europe/Dublin	2010-08-14
+2960954	Whitechurch	Whitechurch		52.31778	-6.96611	P	PPL	IE		L	30			0		46	Europe/Dublin	2010-08-14
+2960955	Whitechurch	Whitechurch		52.10944	-7.74944	P	PPL	IE		M	27			0		62	Europe/Dublin	2010-08-14
+2960956	Whitechurch	Whitechurch		51.98139	-8.51389	P	PPL	IE		M	04			0		152	Europe/Dublin	2010-08-14
+2960957	White Castle	White Castle		55.13333	-7.16667	P	PPL	IE		U	06			0		-9999	Europe/Dublin	1993-12-27
+2960958	White Ball Head	White Ball Head		51.59667	-10.055	T	CAPE	IE		M	04			0		-9999	Europe/Dublin	2010-08-14
+2960959	Whiddy Island	Whiddy Island	Whiddy Island	51.70139	-9.49972	T	ISL	IE		M	04			0		-9999	Europe/Dublin	2010-08-14
+2960960	Wheelam Cross Roads	Wheelam Cross Roads		53.20278	-6.87083	P	PPL	IE		L	12			0		157	Europe/Dublin	2010-08-14
+2960961	Whale Head	Whale Head		55.0525	-7.57389	T	CAPE	IE		U	06			0		1	Europe/Dublin	2010-08-14
+2960962	Wexford Harbour	Wexford Harbour	Wexford Harbour	52.33833	-6.40417	H	HBR	IE		L	30			0		-9999	Europe/Dublin	2010-08-14
+2960963	Loch Garman	Loch Garman	Contae Loch Garman,County Wexford,Wexford	52.5	-6.66667	A	ADM2	IE		L	30			121122		76	Europe/Dublin	2010-08-14
+2960964	Loch Garman	Loch Garman	Loch Garman,Wexford	52.33417	-6.4575	P	PPLA2	IE		L	30			17708		-9999	Europe/Dublin	2010-08-14
+2960965	West Village	West Village		51.88333	-8.6	P	PPL	IE		M	04			0		66	Europe/Dublin	2010-08-14
+2960966	West Town	West Town	Baile Thiar,West Town	55.26528	-8.22583	P	PPL	IE		U	06			0		-9999	Europe/Dublin	2010-08-14
+2960967	West Sister	West Sister	Beenhenry,West Sister	52.2	-10.43333	T	HLL	IE	IE	M	11			0		61	Europe/Dublin	2010-08-14
+2960968	Westport House	Westport House		53.8	-9.51667	S	EST	IE		C	20			0		108	Europe/Dublin	1993-12-27
+2960969	Westport Bay	Westport Bay		53.8	-9.63333	H	BAY	IE		C	20			0		-9999	Europe/Dublin	1993-12-27
+2960970	Cathair na Mart	Cathair na Mart	Westport	53.8	-9.51667	P	PPL	IE		C	20			6200		108	Europe/Dublin	2010-08-14
+2960971	Westmoreland Fort	Westmoreland Fort	Fort Westmoreland,Westmoreland Fort	51.83333	-8.28333	S	FT	IE	IE	M	04			0		-9999	Europe/Dublin	2010-08-14
+2960972	An Iarmh�	An Iarmhi	Contae na hIarmhi,Contae na hIarmh�,County Westmeath,Westmeath	53.5	-7.5	A	ADM2	IE		L	29			75298		144	Europe/Dublin	2010-08-14
+2960973	Westland Row Station	Westland Row Station	Westland Row Passenger Station	53.35	-6.25	S	RSTN	IE	IE	L	07			0		1	Europe/Dublin	2010-08-14
+2960974	Westland House	Westland House		53.785	-6.89694	S	EST	IE		L	21			0		79	Europe/Dublin	2010-08-14
+2960975	Westfield	Westfield		51.55	-9.38333	S	EST	IE		M	04			0		92	Europe/Dublin	1993-12-27
+2960976	West Cove	West Cove		51.77917	-10.07694	P	PPL	IE		M	11			0		134	Europe/Dublin	2010-08-14
+2960977	West Court	West Court		52.54917	-7.39389	S	EST	IE		L	13			0		71	Europe/Dublin	2010-08-14
+2960978	Wells House	Wells House	Wells,Wells House	52.52833	-6.34444	S	EST	IE	IE	L	30			0		58	Europe/Dublin	2010-08-14
+2960979	Wellmount House	Wellmount House		53.10472	-8.22778	S	BLDG	IE		C	10			0		77	Europe/Dublin	2010-08-14
+2960980	Wellingtonbridge	Wellingtonbridge	Wellingtonbridge	52.26333	-6.77833	P	PPL	IE		L	30			0		5	Europe/Dublin	2010-08-14
+2960981	Welchtown	Welchtown		54.81861	-7.87083	P	PPL	IE		U	06			0		84	Europe/Dublin	2010-08-14
+2960982	Webbsborough	Webbsborough	Webbsborough,Webbsborough House	52.75083	-7.23111	S	BLDG	IE	IE	L	13			0		128	Europe/Dublin	2010-08-14
+2960983	Weatherstown	Weatherstown		52.37361	-7.03944	P	PPL	IE		L	13			0		108	Europe/Dublin	2010-08-14
+2960984	Waterville	Waterville	An Coirean,An Coire�n,Waterville	51.83333	-10.16667	P	PPL	IE		M	11			0		14	Europe/Dublin	2010-08-14
+2960985	Waterstown House	Waterstown House	Waterstown,Waterstown House	53.46667	-7.85	S	EST	IE		L	29			0		136	Europe/Dublin	2010-08-10
+2960986	Waterloo	Waterloo		53.93333	-7.13028	P	PPL	IE		U	02			0		171	Europe/Dublin	2010-08-14
+2960987	Watergrasshill	Watergrasshill	Cnocan na Biolrai,Cnoc�n na Biolra�,Watergrasshill	52.01139	-8.34417	P	PPL	IE		M	04			0		156	Europe/Dublin	2010-08-14
+2960988	Waterford North Station	Waterford North Station	Waterford,Waterford North Station	52.26667	-7.11667	S	RSTN	IE	IE	L	13			0		58	Europe/Dublin	2010-08-14
+2960989	Waterford Manor Station	Waterford Manor Station	Manor Station,Waterford Manor Station	52.25	-7.11667	S	RSTN	IE	IE	M	27			0		17	Europe/Dublin	2010-08-14
+2960990	Waterford Harbour	Waterford Harbour		52.17917	-6.93056	H	ESTY	IE		00				0		-9999	Europe/Dublin	1998-01-09
+2960991	Port L�irge	Port Lairge	Contae Phort Lairge,Contae Phort L�irge,County Waterford,Waterford	52.25	-7.5	A	ADM2	IE		M	27			104132		178	Europe/Dublin	2010-08-14
+2960992	Port L�irge	Port Lairge	Port Lairge,Port L�irge,Uoterford,Waterford,Waterford city,u~otafodo,\u0423\u043e\u0442\u0435\u0440\u0444\u043e\u0440\u0434,\u30a6\u30a9\u30fc\u30bf\u30fc\u30d5\u30a9\u30fc\u30c9	52.25833	-7.11194	P	PPLA2	IE		M	27			47904		8	Europe/Dublin	2010-08-14
+2960993	Waterfall	Waterfall		51.85944	-8.55917	P	PPL	IE		M	04			0		77	Europe/Dublin	2010-08-14
+2960994	Waterdale	Waterdale	Waterdale,Waterdale House	53.36667	-8.95	S	BLDG	IE		C	10			0		21	Europe/Dublin	2010-08-10
+2960995	Watch House Village	Watch House Village	Watch House,Watch House Village	52.69083	-6.63972	P	PPL	IE	IE	L	30			0		76	Europe/Dublin	2010-08-14
+2960996	Warrenstown House	Warrenstown House		53.51667	-6.61667	S	EST	IE		L	21			0		92	Europe/Dublin	1993-12-27
+2960997	Warrens Grove	Warrens Grove		51.85	-8.83333	S	EST	IE		M	04			0		79	Europe/Dublin	2010-08-14
+2960998	Warrens Court	Warrens Court		51.85472	-8.91278	S	EST	IE		M	04			0		146	Europe/Dublin	2010-08-14
+2960999	War Hill	War Hill		53.13778	-6.25389	T	MT	IE		L	31			0	685	439	Europe/Dublin	2010-08-14
+2961000	Hill of Ward	Hill of Ward	Hill of Ward	53.62444	-6.88611	T	HLL	IE		L	21			0		79	Europe/Dublin	2010-08-14
+2961001	Ward	Ward		53.43306	-6.33333	P	PPL	IE		L	07			0		66	Europe/Dublin	2010-08-14
+2961002	Walterstown	Walterstown		53.61278	-6.55306	P	PPL	IE		L	21			0		79	Europe/Dublin	2010-08-14
+2961003	Walshtown	Walshtown		51.98306	-8.18722	P	PPL	IE		M	04			0		130	Europe/Dublin	2010-08-14
+2961004	Walshpool Lough	Walshpool Lough		53.8	-9.18333	H	LK	IE		C	20			0		65	Europe/Dublin	1993-12-27
+2961005	Virginia Road	Virginia Road	Virginia Road,Virginia Road Station	53.75	-7.01667	S	RSTN	IE		L	21			0		160	Europe/Dublin	2010-08-10
+2961006	Virginia	Virginia	Achadh an Iuir,Achadh an I�ir,Virginia	53.83389	-7.07556	P	PPL	IE		U	02			0		120	Europe/Dublin	2010-08-14
+2961007	Violetstown House	Violetstown House		53.48333	-7.28333	S	EST	IE		L	29			0		124	Europe/Dublin	1993-12-27
+2961008	Violet Hill	Violet Hill		52.74444	-7.56083	S	EST	IE		L	13			0		142	Europe/Dublin	2010-08-14
+2961009	Vinegar Hill	Vinegar Hill		52.50417	-6.54611	T	HLL	IE		L	30			0		61	Europe/Dublin	2010-08-14
+2961010	Villierstown	Villierstown	An Baile Nua,Villierstown	52.08889	-7.85306	P	PPL	IE		M	27			0		75	Europe/Dublin	2010-08-14
+2961011	Vicarstown Cross Roads	Vicarstown Cross Roads	Vicarstown,Vicarstown Cross Roads	51.93333	-8.65	P	PPL	IE	IE	M	04			0		76	Europe/Dublin	2010-08-14
+2961012	Vicarstown	Vicarstown		53.04917	-7.08861	P	PPL	IE		L	15			0		71	Europe/Dublin	2010-08-14
+2961013	Ventry Harbour	Ventry Harbour		52.12278	-10.35583	H	HBR	IE		M	11			0		-9999	Europe/Dublin	2010-08-14
+2961014	Ventry	Ventry	Ceann Tra,Ceann Tr�,Ventry	52.1325	-10.36694	P	PPL	IE		M	11			0		1	Europe/Dublin	2010-08-14
+2961015	Velvetstown House	Velvetstown House		52.25	-8.66667	S	EST	IE		M	04			0		114	Europe/Dublin	1993-12-27
+2961016	Lough Vearty	Lough Vearty	Lough Vearty	54.53333	-8	H	LK	IE		00				0		177	Europe/Dublin	1998-02-19
+2961017	Lough Beagh	Lough Beagh	Lough Beagh,Lough Veagh	55.03833	-7.97167	H	LK	IE	IE	U	06			0		70	Europe/Dublin	2010-08-14
+2961018	Vartry River	Vartry River		53.00278	-6.05528	H	STM	IE		L	31			0		-9999	Europe/Dublin	2010-08-14
+2961019	Valencia River	Valencia River	Valencia River,Valentia River	51.93222	-10.27806	H	STM	IE	IE	M	11			0		-9999	Europe/Dublin	2010-08-14
+2961020	Valencia Island	Valencia Island	Valencia Island,Valentia Island	51.90833	-10.37194	T	ISL	IE	IE	M	11			0		28	Europe/Dublin	2010-08-14
+2961021	Valencia Harbour Station	Valencia Harbour Station	Valencia Harbour,Valencia Harbour Station,Valentia Harbour	51.91667	-10.28333	S	RSTN	IE	IE	M	11			0		-9999	Europe/Dublin	2010-08-14
+2961022	Valencia Harbour	Valencia Harbour	Valencia Harbour,Valentia Harbour	51.93	-10.30306	H	HBR	IE		M	11			0		35	Europe/Dublin	2010-08-14
+2961023	Urris Hills	Urris Hills		55.22389	-7.51889	T	MTS	IE		U	06			0		269	Europe/Dublin	2010-08-14
+2961024	Urrin River	Urrin River		52.49278	-6.56722	H	STM	IE		L	30			0		52	Europe/Dublin	2010-08-14
+2961025	Urlingford	Urlingford	Ath na nUrlainn,Urlingford,�th na nUrlainn	52.72056	-7.5825	P	PPL	IE		L	13			0		143	Europe/Dublin	2010-08-14
+2961026	Urlaur Lough	Urlaur Lough		53.83333	-8.73333	H	LK	IE		C	20			0		54	Europe/Dublin	1993-12-27
+2961027	Uregare House	Uregare House	Uregare,Uregare House	52.45	-8.56667	S	EST	IE		M	16			0		75	Europe/Dublin	2010-08-10
+2961028	Urbalreagh	Urbalreagh		55.33639	-7.28528	P	PPL	IE		U	06			0		76	Europe/Dublin	2010-08-14
+2961029	Upton House	Upton House		51.79917	-8.68833	S	EST	IE		M	04			0		76	Europe/Dublin	2010-08-14
+2961030	Upton	Upton		51.79222	-8.67333	P	PPL	IE		M	04			0		76	Europe/Dublin	2010-08-14
+2961031	Upper Court	Upper Court		52.72611	-7.40667	S	EST	IE		L	13			0		128	Europe/Dublin	2010-08-14
+2961032	Upperchurch	Upperchurch	Upperchurch	52.70333	-8.0175	P	PPL	IE		M	26			0		289	Europe/Dublin	2010-11-04
+2961033	Unshin River	Unshin River		54.19139	-8.48167	H	STM	IE		C	25			0		59	Europe/Dublin	2010-08-14
+2961034	Lough Unshin	Lough Unshin		54.52028	-8.07917	H	LK	IE		U	06			0		403	Europe/Dublin	2010-08-14
+2961035	Unionhall	Unionhall	Breantra,Br�antr�,Unionhall	51.55	-9.13333	P	PPL	IE		M	04			0		28	Europe/Dublin	2010-08-14
+2961036	Umrycam	Umrycam		55.18333	-7.7	P	PPL	IE		U	06			0		-9999	Europe/Dublin	2010-08-14
+2961037	Ummeryroe	Ummeryroe		54.11667	-8.28333	P	PPL	IE		C	25			0		142	Europe/Dublin	1993-12-27
+2961038	Umma House	Umma House		53.45	-7.7	S	EST	IE		L	29			0		76	Europe/Dublin	1993-12-27
+2961039	Umfin Island	Umfin Island		55.10194	-8.36583	T	ISL	IE		U	06			0		-9999	Europe/Dublin	2010-08-14
+2961040	Ullard	Ullard		52.57972	-6.93083	P	PPL	IE		L	13			0		27	Europe/Dublin	2010-08-14
+2961041	Lough Ugga Beg	Lough Ugga Beg		53.27194	-9.4325	H	LK	IE		C	10			0		74	Europe/Dublin	2010-08-14
+2961042	Tyrrelstown House	Tyrrelstown House		53.4	-6.4	S	EST	IE		L	07			0		36	Europe/Dublin	2010-08-14
+2961043	Tyrrellspass	Tyrrellspass	Bealach an Tirialaigh,Tyrrellspass,Tyrrelspass	53.38889	-7.37361	P	PPL	IE		L	29			0		133	Europe/Dublin	2010-08-14
+2961044	Tyredagh Castle	Tyredagh Castle	Tyredach Castle,Tyredagh Castle	52.88472	-8.79861	S	RUIN	IE	IE	M	03			0		66	Europe/Dublin	2010-08-14
+2961045	Tyrcallen	Tyrcallen		54.82528	-7.74417	S	EST	IE		U	06			0		82	Europe/Dublin	2010-08-14
+2961046	Tynagh	Tynagh	Tine,Tynagh,T�ne	53.14944	-8.3725	P	PPL	IE	IE	C	10			0		75	Europe/Dublin	2010-08-14
+2961047	Tyholland Bridge	Tyholland Bridge	Tyholland,Tyholland Bridge	54.25	-6.9	P	PPL	IE	IE	00				0		65	Europe/Dublin	2010-08-10
+2961048	Two Rock Mountain	Two Rock Mountain		53.23472	-6.24167	T	MT	IE		L	07			0	517	453	Europe/Dublin	2010-08-14
+2961049	Twomileditch	Twomileditch		53.30389	-8.99778	P	PPL	IE		C	10			0		3	Europe/Dublin	2010-08-14
+2961050	Twomileborris	Twomileborris		52.67167	-7.72	P	PPL	IE		M	26			0		124	Europe/Dublin	2010-08-14
+2961051	The Twelve Pins	The Twelve Pins	Beanna Beola,Benna Beola,The Twelve Bens,The Twelve Pins,Twelve Pins of Bennebeola	53.5	-9.83333	T	MTS	IE	IE	C	10			0		402	Europe/Dublin	2010-08-14
+2961052	Tuskar Rock	Tuskar Rock	Tuskar Rock	52.19889	-6.19833	T	RK	IE		L	30			0		-9999	Europe/Dublin	2010-08-14
+2961053	Turvey House	Turvey House		53.49667	-6.17556	S	EST	IE		L	07			0		1	Europe/Dublin	2010-08-14
+2961054	Turners Rock	Turners Rock		51.79222	-9.58111	T	MT	IE		M	04			0		233	Europe/Dublin	2010-08-14
+2961055	Turloughmore	Turloughmore		53.37472	-8.86472	P	PPL	IE		C	10			0		27	Europe/Dublin	2010-08-14
+2961056	Turlough	Turlough	Turlach,Turlough	53.88333	-9.21667	P	PPL	IE	IE	C	20			0		38	Europe/Dublin	2010-08-14
+2961057	Turlough	Turlough		53.09222	-9.06528	P	PPL	IE		M	03			0		68	Europe/Dublin	2010-08-14
+2961058	Turkstown	Turkstown		52.32333	-7.30389	P	PPL	IE		L	13			0		13	Europe/Dublin	2010-08-14
+2961059	Turbotston	Turbotston	Turbotston,Turbotstown,Turbotstown House	53.7	-7.35	S	EST	IE	IE	L	29			0		79	Europe/Dublin	2010-08-14
+2961060	Turbot Island	Turbot Island	Talbot Island,Turbot Island	53.50194	-10.14806	T	ISL	IE	IE	C	10			0		-9999	Europe/Dublin	2010-08-14
+2961061	Tulsk	Tulsk	Tuilsce,Tulsk	53.78333	-8.25	P	PPL	IE	IE	C	24			0		75	Europe/Dublin	2010-08-14
+2961062	Tullyvin House	Tullyvin House	Tullyvin,Tullyvin House	54.05	-7.15	S	HSEC	IE	IE	U	02			0		76	Europe/Dublin	2010-08-14
+2961063	Tullyvin	Tullyvin		54.04833	-7.14472	P	PPL	IE		U	02			0		76	Europe/Dublin	2010-08-14
+2961064	Tullynascreen	Tullynascreen		54.17472	-8.28167	P	PPL	IE		C	14			0		174	Europe/Dublin	2010-08-14
+2961065	Tullynahinnera	Tullynahinnera	Tullynahinera,Tullynahinnera	54.08333	-6.8	P	PPL	IE	IE	00				0		152	Europe/Dublin	2010-08-10
+2961066	Tullymurray	Tullymurray		54.13333	-8.1	P	PPL	IE		C	14			0		58	Europe/Dublin	1993-12-27
+2961067	Tullymore	Tullymore	Tullamore,Tullymore	54.52111	-8.17806	P	PPL	IE	IE	U	06			0		67	Europe/Dublin	2010-08-14
+2961068	Tully Lough	Tully Lough	Tally Lough,Tully Lough	53.58972	-9.97972	H	LK	IE		C	10			0		15	Europe/Dublin	2010-08-14
+2961069	Tullylease	Tullylease		52.3	-8.95	P	PPL	IE		M	04			0		155	Europe/Dublin	1993-12-27
+2961070	Tullyleague	Tullyleague		52.53611	-9.30028	P	PPL	IE		M	16			0		148	Europe/Dublin	2010-08-14
+2961071	Tullyglass House	Tullyglass House	Tullyglass,Tullyglass House	51.76	-8.86361	S	BLDG	IE	IE	M	04			0		110	Europe/Dublin	2010-08-14
+2961072	Tullycreen Upper	Tullycreen Upper	Tullycreen,Tullycreen Upper,Tullycrine	52.68389	-9.35583	P	PPL	IE	IE	M	03			0		27	Europe/Dublin	2010-08-14
+2961073	Tullycreen Lower	Tullycreen Lower		52.66833	-9.32722	P	PPL	IE		M	03			0		24	Europe/Dublin	2010-08-14
+2961074	Tullycanna	Tullycanna		52.2675	-6.66861	P	PPL	IE		L	30			0		35	Europe/Dublin	2010-08-14
+2961075	Tullybrook	Tullybrook		54.63194	-8.07889	P	PPL	IE		U	06			0		53	Europe/Dublin	2010-08-14
+2961076	Tullyard	Tullyard		53.58194	-6.79333	S	EST	IE		L	21			0		77	Europe/Dublin	2010-08-14
+2961077	Tullyallen	Tullyallen	Tullyallen	53.73611	-6.42278	P	PPL	IE		L	19			0		60	Europe/Dublin	2010-08-14
+2961078	Tully	Tully		53.23333	-9.48333	P	PPL	IE		C	10			0		-9999	Europe/Dublin	1993-12-27
+2961079	Tullow	Tullow	An Tulach,Tullow	52.80028	-6.73694	P	PPL	IE		L	01			2424		86	Europe/Dublin	2010-08-14
+2961080	Tullira Castle	Tullira Castle		53.13611	-8.78444	S	HSEC	IE		C	10			0		33	Europe/Dublin	2010-08-14
+2961081	Tullig Point	Tullig Point		52.61417	-9.80417	T	PT	IE		M	03			0		2	Europe/Dublin	2010-08-14
+2961082	Tullig	Tullig		52.61472	-9.77611	P	PPL	IE		M	03			0		30	Europe/Dublin	2010-08-14
+2961083	Tullig	Tullig		52.11278	-9.85444	P	PPL	IE		M	11			0		8	Europe/Dublin	2010-08-14
+2961084	Tullaroan	Tullaroan	Tullaroan	52.66222	-7.43639	P	PPL	IE		L	13			0		152	Europe/Dublin	2010-08-14
+2961085	Tullamore River	Tullamore River	Clonmore,Tullamore River	53.27722	-7.5775	H	STM	IE	IE	L	23			0		71	Europe/Dublin	2010-08-14
+2961086	Tullamore	Tullamore	Tulach Mhor,Tulach Mh�r	53.27389	-7.48889	P	PPLA2	IE		L	23			11575		73	Europe/Dublin	2010-08-14
+2961087	Tullamore	Tullamore		52.49167	-9.48778	P	PPL	IE		M	11			0		68	Europe/Dublin	2010-08-14
+2961088	Tullaher Lough	Tullaher Lough	Lough Trullaher,Tullaher Lough	52.69861	-9.54528	H	LK	IE	IE	M	03			0		1	Europe/Dublin	2010-08-14
+2961089	Tullagh Point	Tullagh Point		55.29472	-7.46722	T	PT	IE		U	06			0		2	Europe/Dublin	2010-08-14
+2961090	Tullaghought	Tullaghought		52.42194	-7.36167	P	PPL	IE		L	13			0		169	Europe/Dublin	2010-08-14
+2961091	Tullagher	Tullagher		52.43167	-7.04556	P	PPL	IE		L	13			0		134	Europe/Dublin	2010-08-14
+2961092	Tullagh Bay	Tullagh Bay		55.28639	-7.44361	H	BAY	IE		U	06			0		-9999	Europe/Dublin	2010-08-14
+2961093	Tullaghanoge	Tullaghanoge		53.58889	-6.87222	P	PPL	IE		L	21			0		77	Europe/Dublin	2010-08-14
+2961094	Tullaghan Bay	Tullaghan Bay	Tallaghan Bay,Tullaghan Bay	54.09111	-9.86889	H	BAY	IE	IE	C	20			0		-9999	Europe/Dublin	2010-08-14
+2961095	Tullaghan	Tullaghan		54.46833	-8.33111	P	PPL	IE		C	14			0		1	Europe/Dublin	2010-08-14
+2961096	Tulla	Tulla	An Tulach,Tulla	52.86472	-8.76056	P	PPL	IE		M	03			0		152	Europe/Dublin	2010-08-14
+2961097	Tudenham Park	Tudenham Park		53.46667	-7.35	S	EST	IE		L	29			0		117	Europe/Dublin	1993-12-27
+2961098	Tubber	Tubber	Tobar,Tubber	52.99889	-8.88528	P	PPL	IE		C	10			0		47	Europe/Dublin	2010-08-14
+2961099	Tuam	Tuam	Tuaim,Tuam	53.51667	-8.85	P	PPL	IE		C	10			6130		48	Europe/Dublin	2010-08-14
+2961100	Truskmore	Truskmore		54.37639	-8.37222	T	MT	IE		00				0	646	591	Europe/Dublin	2006-01-16
+2961101	Trusk Lough	Trusk Lough		54.75722	-7.79917	H	LK	IE		U	06			0		161	Europe/Dublin	2010-08-14
+2961102	Trusklieve	Trusklieve		52.62778	-9.76444	P	PPL	IE		M	03			0		73	Europe/Dublin	2010-08-14
+2961103	Trubley Castle	Trubley Castle		53.56667	-6.71667	S	RUIN	IE		L	21			0		57	Europe/Dublin	1993-12-27
+2961104	Trough	Trough		52.73333	-8.61667	P	PPL	IE		M	03			0		79	Europe/Dublin	1993-12-27
+2961105	Triogue River	Triogue River	Tiogue River,Triogue River	53.11389	-7.28972	H	STM	IE	IE	L	15			0		107	Europe/Dublin	2010-08-14
+2961106	Trimoge River	Trimoge River		53.91667	-9.01667	H	STM	IE		C	20			0		30	Europe/Dublin	1993-12-27
+2961107	Baile �tha Troim	Baile Atha Troim	Baile Atha Troim,Baile �tha Troim,Trim	53.555	-6.79167	P	PPL	IE		L	21			1800		62	Europe/Dublin	2010-09-05
+2961108	The Triangle	The Triangle		53.76667	-9.4	P	PPLL	IE		C	20			0		137	Europe/Dublin	2010-08-14
+2961109	Tremone Bay	Tremone Bay		55.26667	-7.06667	H	BAY	IE		U	06			0		1	Europe/Dublin	1993-12-27
+2961110	Tremblestown River	Tremblestown River		53.55417	-6.83167	H	STM	IE		L	21			0		66	Europe/Dublin	2010-08-14
+2961111	Treantagh	Treantagh		54.92667	-7.4875	P	PPL	IE		U	06			0		52	Europe/Dublin	2010-08-14
+2961112	Trawmore Bay	Trawmore Bay		54.18361	-9.9575	H	BAY	IE		C	20			0		-9999	Europe/Dublin	2010-08-14
+2961113	Trawenagh Bay	Trawenagh Bay		54.89333	-8.35417	H	BAY	IE		U	06			0		-9999	Europe/Dublin	2010-08-14
+2961114	Trawbreaga Bay	Trawbreaga Bay	Trawbreaga Bay,Trawbreaga Lough	55.285	-7.29472	H	BAY	IE	IE	U	06			0		-9999	Europe/Dublin	2010-08-14
+2961115	Traverston House	Traverston House		52.7975	-8.16444	S	EST	IE		M	26			0		118	Europe/Dublin	2010-08-14
+2961116	Tranarossan Bay	Tranarossan Bay	Tranarossan Bay,Tranrossan Bay	55.23056	-7.82444	H	BAY	IE	IE	U	06			0		-9999	Europe/Dublin	2010-08-14
+2961117	Tranagh	Tranagh		52.69472	-7.6025	P	PPL	IE		M	26			0		138	Europe/Dublin	2010-08-14
+2961118	Tramore Bay	Tramore Bay		55.18333	-8.03333	H	BAY	IE		U	06			0		-9999	Europe/Dublin	1993-12-27
+2961119	Tramore Bay	Tramore Bay		52.13917	-7.1325	H	BAY	IE		M	27			0		-9999	Europe/Dublin	2010-08-14
+2961120	Tr� Mh�r	Tra Mhor	Tramore	52.16235	-7.15244	P	PPL	IE		M	27			9164		2	Europe/Dublin	2010-08-14
+2961121	Tralong Bay	Tralong Bay		51.55694	-9.05861	H	BAY	IE		M	04			0		-9999	Europe/Dublin	2010-08-14
+2961122	Tralee Bay	Tralee Bay	Tralee Bay	52.27833	-9.93139	H	BAY	IE		M	11			0		-9999	Europe/Dublin	2010-08-14
+2961123	Tr� L�	Tra Li	Tra Li,Tralee,Tr� L�	52.27042	-9.70264	P	PPLA2	IE		M	11			22941	912	48	Europe/Dublin	2010-10-06
+2961124	Tracton	Tracton		51.76139	-8.41639	P	PPL	IE		M	04			0		76	Europe/Dublin	2010-08-14
+2961125	Tracarta	Tracarta		51.51583	-9.19389	P	PPL	IE		M	04			0		6	Europe/Dublin	2010-08-14
+2961126	Trabolgan House	Trabolgan House		51.79972	-8.23583	S	EST	IE		M	04			0		1	Europe/Dublin	2010-08-14
+2961127	Townley Hall	Townley Hall		53.72944	-6.44722	S	EST	IE		L	19			0		54	Europe/Dublin	2010-08-14
+2961128	Towerhill House	Towerhill House		53.71667	-9.2	S	EST	IE		C	20			0		46	Europe/Dublin	1993-12-27
+2961129	Tower	Tower	Tower,Tower Village	51.91667	-8.6	P	PPL	IE		M	04			3099		70	Europe/Dublin	2010-08-14
+2961130	Tourin	Tourin		52.11667	-7.86667	P	PPL	IE		M	27			0		17	Europe/Dublin	1993-12-27
+2961131	Tourig River	Tourig River		51.97778	-7.8625	H	STM	IE		M	04			0		-9999	Europe/Dublin	2010-08-14
+2961132	Tory Sound	Tory Sound		55.21667	-8.2	H	SD	IE		U	06			0		-9999	Europe/Dublin	1993-12-27
+2961133	Toraigh	Toraigh	Toraigh,Tory Island	55.26444	-8.22111	T	ISL	IE		U	06			0		15	Europe/Dublin	2010-08-14
+2961134	The Tor Rocks	The Tor Rocks		55.43333	-7.25	T	RKS	IE		U	06			0		-9999	Europe/Dublin	2010-08-14
+2961135	Torneady Point	Torneady Point	Tornado Point,Torneady Point	55.02167	-8.54028	T	PT	IE	IE	U	06			0		35	Europe/Dublin	2010-08-14
+2961136	Tormore Island	Tormore Island		54.76389	-8.69222	T	ISL	IE		U	06			0		-9999	Europe/Dublin	2010-08-14
+2961137	Torc Mountain	Torc Mountain	Torc Mountain	52	-9.51667	T	MT	IE		M	11			0		189	Europe/Dublin	2010-08-10
+2961138	Torc Cascade	Torc Cascade	Torc Cascade,Tore Cascade	52	-9.5	H	FLLS	IE		M	11			0		378	Europe/Dublin	2010-08-10
+2961139	Toormore Bay	Toormore Bay		51.50194	-9.65583	H	BAY	IE		M	04			0		-9999	Europe/Dublin	2010-08-14
+2961140	Toormore	Toormore		53.79278	-10.03417	L	LCTY	IE		C	20			0		111	Europe/Dublin	2010-08-14
+2961141	Toormore	Toormore		51.52083	-9.65611	P	PPL	IE		M	04			0		-9999	Europe/Dublin	2010-08-14
+2961142	Toormakeady Lodge	Toormakeady Lodge	Toormakeady,Toormakeady Lodge	53.65	-9.38333	S	EST	IE		C	20			0		59	Europe/Dublin	2010-08-10
+2961143	Toormakeady	Toormakeady		53.65	-9.36667	P	PPL	IE		C	20			0		59	Europe/Dublin	1993-12-27
+2961144	Tooreeny	Tooreeny		53	-8.38333	P	PPL	IE		C	10			0		68	Europe/Dublin	2010-08-14
+2961145	Tooreens	Tooreens		51.84278	-10.02861	P	PPL	IE		M	11			0		421	Europe/Dublin	2010-08-14
+2961146	Tooreengarriv	Tooreengarriv		52.16667	-9.28333	P	PPL	IE		M	11			0		241	Europe/Dublin	1993-12-27
+2961147	Tooreendonnell	Tooreendonnell		52.495	-9.22694	P	PPL	IE		M	16			0		152	Europe/Dublin	2010-08-14
+2961148	Tooreencahill	Tooreencahill		52.15	-9.25	P	PPL	IE		M	11			0		181	Europe/Dublin	1993-12-27
+2961149	Tooreenbrien Bridge	Tooreenbrien Bridge	Tooreenbrien,Tooreenbrien Bridge	52.6925	-8.27361	P	PPL	IE	IE	M	26			0		288	Europe/Dublin	2010-11-04
+2961150	Tooreen	Tooreen		53.66667	-9.28333	P	PPL	IE		C	20			0		49	Europe/Dublin	1993-12-27
+2961151	Tooreen	Tooreen		51.55	-9.83333	P	PPL	IE		M	04			0		-9999	Europe/Dublin	1993-12-27
+2961152	Tooravoola	Tooravoola		53.05667	-8.64028	P	PPL	IE		C	10			0		146	Europe/Dublin	2010-08-14
+2961153	Tooraneena	Tooraneena		52.20222	-7.715	P	PPL	IE		M	27			0		152	Europe/Dublin	2010-08-14
+2961154	Toor	Toor		51.8	-10.16667	P	PPL	IE		M	11			0		39	Europe/Dublin	2010-08-14
+2961155	Toon River	Toon River		51.88333	-9.01667	H	STM	IE		M	04			0		104	Europe/Dublin	1993-12-27
+2961156	Toonagh House	Toonagh House		52.88778	-9.02833	S	EST	IE		M	03			0		50	Europe/Dublin	2010-08-14
+2961157	Toomyvara	Toomyvara	Toomyvara,Tuaim Ui Mheara,Tuaim U� Mhe�ra	52.84972	-8.03222	P	PPL	IE		M	38			0		157	Europe/Dublin	2010-11-04
+2961158	Toomona House	Toomona House		53.76667	-8.28333	S	EST	IE		C	24			0		78	Europe/Dublin	1993-12-27
+2961159	Toomard	Toomard		53.53333	-8.46667	P	PPL	IE		C	10			0		73	Europe/Dublin	1993-12-27
+2961160	Tooban Junction	Tooban Junction		55.05	-7.43333	S	RSTN	IE		U	06			0		55	Europe/Dublin	1993-12-27
+2961161	Tonlegee House	Tonlegee House	Tonlegee,Tonlegee House	52.9775	-6.98917	S	EST	IE	IE	L	15			0		59	Europe/Dublin	2010-08-14
+2961162	Tonduff	Tonduff		55.2	-7.51667	P	PPL	IE		U	06			0		62	Europe/Dublin	1993-12-27
+2961163	Tomies Mountain	Tomies Mountain		52.01667	-9.61667	T	MT	IE		M	11			0		432	Europe/Dublin	1993-12-27
+2961164	Tomhaggard	Tomhaggard		52.21194	-6.51778	P	PPL	IE		L	30			0		11	Europe/Dublin	2010-08-14
+2961165	Tuamgraney	Tuamgraney	Tomgraney,Tuaim Greine,Tuaim Gr�ine,Tuamgraney	52.89722	-8.53861	P	PPL	IE		M	03			0		75	Europe/Dublin	2010-08-14
+2961166	Tombrack	Tombrack		52.60278	-6.5475	P	PPL	IE		L	30			0		71	Europe/Dublin	2010-08-14
+2961167	Tomacurry	Tomacurry		52.55333	-6.56	P	PPL	IE		L	30			0		56	Europe/Dublin	2010-08-14
+2961168	Tolka River	Tolka River	An Tulca,Tolka River	53.34972	-6.23889	H	STM	IE	IE	00				0		1	Europe/Dublin	2010-08-10
+2961169	Toghermore	Toghermore		53.48333	-8.81667	S	EST	IE		C	10			0		43	Europe/Dublin	1993-12-27
+2961170	The Togher	The Togher		53.07972	-7.19972	P	PPL	IE		L	15			0		134	Europe/Dublin	2010-08-14
+2961171	Togher	Togher		53.84278	-6.30278	P	PPL	IE		L	19			0		21	Europe/Dublin	2010-08-14
+2961172	Togher	Togher		53.08167	-6.52	P	PPL	IE		L	31			0		280	Europe/Dublin	2010-08-14
+2961173	Roundwood	Roundwood	An Tochar,An T�char,Roundwood,Togher	53.05861	-6.22611	P	PPL	IE		L	31			0		258	Europe/Dublin	2010-08-14
+2961174	Togher	Togher		51.8775	-8.49167	P	PPL	IE		M	04			0		56	Europe/Dublin	2010-08-14
+2961175	Togher	Togher		51.78111	-9.15556	P	PPL	IE		M	04			0		145	Europe/Dublin	2010-08-14
+2961176	Toem	Toem		52.57667	-8.19611	P	PPL	IE		M	26			0		96	Europe/Dublin	2010-08-14
+2961177	Toe Head Bay	Toe Head Bay		51.49111	-9.25167	H	BAY	IE		M	04			0		-9999	Europe/Dublin	2010-08-14
+2961178	Toe Head	Toe Head		51.48583	-9.23556	T	CAPE	IE		M	04			0		3	Europe/Dublin	2010-08-14
+2961179	Tobertynan House	Tobertynan House	Tobertynan,Tobertynan House	53.48528	-6.8575	S	EST	IE	IE	L	21			0		79	Europe/Dublin	2010-08-14
+2961180	Toberscanavan	Toberscanavan		54.16694	-8.48556	P	PPL	IE		C	25			0		66	Europe/Dublin	2010-08-14
+2961181	Toberdan	Toberdan		53.53333	-8.06667	P	PPL	IE		C	24			0		76	Europe/Dublin	1993-12-27
+2961182	Tobercurry	Tobercurry	Tobar an Choire,Tobercurry,Tubbercurry	54.05	-8.73333	P	PPL	IE		C	25			0		92	Europe/Dublin	2010-08-14
+2961183	Tober	Tober		54.21667	-8.00194	P	PPL	IE		U	02			0		158	Europe/Dublin	2010-08-14
+2961184	Tober	Tober		53.375	-7.65611	P	PPL	IE		L	23			0		73	Europe/Dublin	2010-08-14
+2961185	Tivoli House	Tivoli House		51.9	-8.43333	S	EST	IE		M	04			0		27	Europe/Dublin	2010-08-14
+2961186	Tirraboy	Tirraboy		55.25	-7.15	P	PPL	IE		U	06			0		25	Europe/Dublin	1993-12-27
+2961187	Tirneevin	Tirneevin		53.06417	-8.88306	P	PPL	IE		C	10			0		44	Europe/Dublin	2010-08-14
+2961188	Tiragarvan	Tiragarvan		53.98333	-6.76667	P	PPL	IE		00				0		99	Europe/Dublin	1993-12-27
+2961189	Tipperary South Riding	Tipperary South Riding	South Riding,Tipperary County South Riding	52.41667	-7.83333	A	ADM2	IE		00	40			80276		151	Europe/Dublin	2010-08-14
+2961190	Tipperary North Riding	Tipperary North Riding	North Riding,Tipperary County North Riding	52.75	-7.83333	A	ADM2	IE		00	38			61926		104	Europe/Dublin	2010-08-14
+2961191	County Tipperary	County Tipperary	Contae Thiobraid Arann,Contae Thiobraid �rann,County Tipperary,Tiobraid Arann,Tiobraid �rann,Tipperary	52.66667	-7.83333	A	ADMD	IE	IE	M	26			0		93	Europe/Dublin	2010-11-04
+2961192	Tiobraid �rann	Tiobraid Arann	Tipperary	52.47333	-8.15583	P	PPL	IE		M	26			4976		166	Europe/Dublin	2010-08-14
+2961193	Tintrim House	Tintrim House		52.96417	-8.34556	S	EST	IE		M	03			0		56	Europe/Dublin	2010-08-14
+2961194	Tintern Abbey	Tintern Abbey		52.23528	-6.83333	S	MSTY	IE		L	30			0		3	Europe/Dublin	2010-08-14
+2961195	Tinny Park	Tinny Park		53.68333	-8.38333	P	PPL	IE		C	24			0		76	Europe/Dublin	1993-12-27
+2961196	Tinnaslatty House	Tinnaslatty House		52.8	-7.41667	S	RUIN	IE		L	13			0		154	Europe/Dublin	2010-08-14
+2961197	Timnapark House	Timnapark House	Timnapark House,Tinnapark,Tinnapark House	53.1	-6.11667	S	HSEC	IE	IE	L	31			0		114	Europe/Dublin	2010-08-14
+2961198	Tinnahinch	Tinnahinch		52.53861	-6.94806	P	PPL	IE		L	01			0		28	Europe/Dublin	2010-08-14
+2961199	Tinnaglogh	Tinnaglogh		52.26806	-6.89722	P	PPL	IE		L	30			0		64	Europe/Dublin	2010-08-14
+2961200	Tinkerslane	Tinkerslane		53.505	-6.98111	P	PPL	IE		L	21			0		84	Europe/Dublin	2010-08-14
+2961201	Tincoora	Tincoora		52.1	-8.86667	P	PPL	IE		M	04			0		156	Europe/Dublin	1993-12-27
+2961202	Tinarana House	Tinarana House	Tinarana,Tinarana House	52.86444	-8.44917	S	EST	IE	IE	M	03			0		55	Europe/Dublin	2010-08-14
+2961203	Tinalira	Tinalira		52.18472	-7.72806	P	PPL	IE		M	27			0		133	Europe/Dublin	1997-12-15
+2961204	Tinahely	Tinahely	Tigh na hEille,Tigh na h�ille,Tinahely	52.79667	-6.46333	P	PPL	IE	IE	L	31			0		121	Europe/Dublin	2010-08-14
+2961205	Timoney Park	Timoney Park	Timaney Park,Timoney Park	52.91139	-7.72333	S	EST	IE	IE	M	26			0		153	Europe/Dublin	2010-11-04
+2961206	Timolin	Timolin		52.9825	-6.81167	P	PPL	IE		L	12			0		138	Europe/Dublin	2010-08-14
+2961207	Timoleague	Timoleague	Tigh Molaige,Timoleague	51.64222	-8.77306	P	PPL	IE	IE	M	04			0		25	Europe/Dublin	2010-08-14
+2961208	Timahoe	Timahoe		53.335	-6.83861	P	PPL	IE		L	12			0		92	Europe/Dublin	2010-08-14
+2961209	Timahoe	Timahoe	Tigh Mochua,Timahoe	52.95833	-7.20528	P	PPL	IE	IE	L	15			0		152	Europe/Dublin	2010-08-14
+2961210	Tiltinbane	Tiltinbane	Tiltinbane	54.21972	-7.85778	T	MT	IE		00				0	594	453	Europe/Dublin	2006-01-16
+2961211	Tievemore House	Tievemore House		54.59444	-7.77472	S	HSEC	IE		U	06			0		173	Europe/Dublin	2010-08-14
+2961212	Tievemore	Tievemore		54.58333	-7.76667	P	PPL	IE		00				0		330	Europe/Dublin	1993-12-27
+2961213	Tiaquin	Tiaquin		53.35972	-8.63333	P	PPL	IE		C	10			0		74	Europe/Dublin	2010-08-14
+2961214	Durlas	Durlas	Thurles	52.68194	-7.80222	P	PPL	IE		M	26			7588		98	Europe/Dublin	2010-08-14
+2961215	The Three Sisters	The Three Sisters		52.20167	-10.42194	T	HLLS	IE		M	11			0		-9999	Europe/Dublin	2010-08-14
+2961216	The Pigeons	The Pigeons	The Pigeons,Three Jolly Pigeons	53.5	-7.81667	P	PPL	IE	IE	L	29			0		77	Europe/Dublin	2010-08-14
+2961217	Threecastles House	Threecastles House	Three Castles,Threecastles House	52.71	-7.32	S	EST	IE	IE	L	13			0		67	Europe/Dublin	2010-08-14
+2961218	Three Castle Head	Three Castle Head		51.48278	-9.83667	T	CAPE	IE		M	04			0		-9999	Europe/Dublin	2010-08-14
+2961219	Thomastown Park	Thomastown Park	Thomastown House,Thomastown Park	53.36583	-8.07333	S	EST	IE	IE	C	24			0		123	Europe/Dublin	2010-08-14
+2961220	Thomaston Park	Thomaston Park	Thomaston Park,Thomastown House,Thomastown Park	53.13389	-7.79583	S	EST	IE	IE	L	23			0		76	Europe/Dublin	2010-08-14
+2961221	Thomastown Castle	Thomastown Castle		53.93333	-6.58333	S	EST	IE		00				0		44	Europe/Dublin	1993-12-27
+2961222	Thomastown Castle	Thomastown Castle		52.48972	-8.03222	S	EST	IE		M	26			0		79	Europe/Dublin	2010-08-14
+2961223	Thomastown	Thomastown		53.39639	-6.88417	P	PPL	IE		L	12			0		86	Europe/Dublin	2010-08-14
+2961224	Thomastown	Thomastown	Baile Mhic Andain,Baile Mhic And�in,Thomastown	52.52667	-7.13722	P	PPL	IE		L	13			1603		52	Europe/Dublin	2010-08-14
+2961225	Thomastown	Thomastown	Thomastown,Thomastown Castle	52.49417	-8.02444	P	PPL	IE	IE	M	26			0		79	Europe/Dublin	2010-11-04
+2961226	Thomas Street	Thomas Street		53.46667	-8.21667	P	PPL	IE		C	24			0		76	Europe/Dublin	1993-12-27
+2961227	Tervoe House	Tervoe House		52.64944	-8.71139	S	EST	IE		M	16			0		3	Europe/Dublin	2010-08-14
+2961228	Terryglass	Terryglass	Terryglass	53.05028	-8.19889	P	PPL	IE		M	26			0		55	Europe/Dublin	2010-11-04
+2961229	Termon River	Termon River	Termon River	54.53333	-7.85	H	STM	IE		00				0		68	Europe/Dublin	1998-02-19
+2961230	Termon Hill	Termon Hill		54.10333	-10.09611	T	HLL	IE		C	20			0	105	2	Europe/Dublin	2010-08-14
+2961231	Termonfeckin	Termonfeckin	Tearmann Feichin,Tearmann Feich�n,Termonfeckin	53.76333	-6.26778	P	PPL	IE		L	19			0		3	Europe/Dublin	2010-08-14
+2961232	Termon Cottage	Termon Cottage		53.04806	-9.06056	S	HSEC	IE		M	03			0		151	Europe/Dublin	2010-08-14
+2961233	Termoncarragh	Termoncarragh		54.25139	-10.06361	P	PPL	IE		C	20			0		57	Europe/Dublin	2010-08-14
+2961234	Termonbarry	Termonbarry		53.75	-7.91667	P	PPL	IE		00				0		66	Europe/Dublin	1993-12-27
+2961235	Terenure	Terenure	Roundtown,Terenure	53.30972	-6.28528	P	PPL	IE	IE	00				0		49	Europe/Dublin	2010-08-10
+2961236	Templetouhy	Templetouhy	Teampall Tuaithe,Templetouhy,Templetuohy	52.78722	-7.71972	P	PPL	IE	IE	M	26			0		147	Europe/Dublin	2010-11-04
+2961237	An Teampall M�r	An Teampall Mor	Templemore	52.79472	-7.83389	P	PPL	IE		M	26			2264		120	Europe/Dublin	2010-08-14
+2961238	Templehouse Lake	Templehouse Lake		54.10333	-8.58917	H	LK	IE		C	25			0		74	Europe/Dublin	2010-08-14
+2961239	Temple House	Temple House		54.11083	-8.59111	S	HSEC	IE		C	25			0		74	Europe/Dublin	2010-08-14
+2961240	Templebreedy	Templebreedy		51.78167	-8.31583	A	PRSH	IE		M	04			0		5	Europe/Dublin	2010-08-14
+2961241	Templeboy	Templeboy		54.24583	-8.8125	P	PPL	IE		C	25			0		43	Europe/Dublin	2010-08-14
+2961242	Templebodan	Templebodan		51.99861	-8.24583	P	PPL	IE		M	04			0		138	Europe/Dublin	2010-08-14
+2961243	Temple	Temple	Oriel Temple,Temple	53.78333	-6.46667	S	EST	IE	IE	00				0		152	Europe/Dublin	2010-08-10
+2961244	Temora House	Temora House		53.16667	-7.75	S	EST	IE		L	23			0		73	Europe/Dublin	2010-08-14
+2961245	Teltown House	Teltown House	Teltown,Teltown House	53.69667	-6.77861	S	BLDG	IE	IE	L	21			0		57	Europe/Dublin	2010-08-14
+2961246	Teeromoyle	Teeromoyle		51.97222	-10.07028	P	PPL	IE		M	11			0		170	Europe/Dublin	2010-08-14
+2961247	Teerelton	Teerelton		51.85139	-8.98444	P	PPL	IE		M	04			0		152	Europe/Dublin	2010-08-14
+2961248	Teeranearagh	Teeranearagh		51.86861	-10.26278	P	PPL	IE		M	11			0		263	Europe/Dublin	2010-08-14
+2961249	Teelin Bay	Teelin Bay	Teelin Bay,Teelin Harbour	54.63333	-8.63333	H	HBR	IE		U	06			0		-9999	Europe/Dublin	2010-08-10
+2961250	Teelin	Teelin		54.63472	-8.64167	P	PPL	IE		U	06			0		15	Europe/Dublin	2010-08-14
+2961251	Tedavnet	Tedavnet		54.29639	-7.01389	P	PPL	IE		U	22			0		138	Europe/Dublin	2010-08-14
+2961252	Tearaght Island	Tearaght Island	Inishteraght,Tearaght Island	52.07528	-10.65667	T	ISL	IE	IE	M	11			0		-9999	Europe/Dublin	2010-08-14
+2961253	River Tay	River Tay		52.125	-7.46222	H	STM	IE		M	27			0		-9999	Europe/Dublin	2010-08-14
+2961254	Lough Tay	Lough Tay	Loch Te,Loch T�,Lough Tay,Luggada Lake	53.10472	-6.26861	H	LK	IE	IE	L	31			0		342	Europe/Dublin	2010-08-14
+2961255	Tawnyinah	Tawnyinah		53.9	-8.76667	P	PPL	IE		C	20			0		135	Europe/Dublin	1993-12-27
+2961256	Tawny Bay	Tawny Bay		54.62167	-8.61861	H	BAY	IE		U	06			0		-9999	Europe/Dublin	2010-08-14
+2961257	Tawnyard Lough	Tawnyard Lough		53.63333	-9.63333	H	LK	IE		C	20			0		74	Europe/Dublin	1993-12-27
+2961258	Tawny	Tawny	Tamney,Tawny	55.2	-7.68333	P	PPL	IE	IE	U	06			0		51	Europe/Dublin	2010-08-14
+2961259	Tawny	Tawny		54.62444	-8.60083	P	PPL	IE		U	06			0		-9999	Europe/Dublin	2010-08-14
+2961260	Tawnawully Mountains	Tawnawully Mountains		54.73528	-7.98333	T	MTS	IE		U	06			0		180	Europe/Dublin	2010-08-14
+2961261	Tawin Island	Tawin Island	Tawin Island,West Tawin	53.21861	-9.03194	T	ISL	IE	IE	C	10			0		1	Europe/Dublin	2010-08-14
+2961262	Taurbeg	Taurbeg		52.23333	-9.13333	T	MT	IE		M	04			0		299	Europe/Dublin	1993-12-27
+2961263	Taur	Taur		52.23333	-9.11667	P	PPL	IE		M	04			0		291	Europe/Dublin	1993-12-27
+2961264	Tarsaghaunmore River	Tarsaghaunmore River		54.06361	-9.785	H	STM	IE		C	20			0		55	Europe/Dublin	2010-08-14
+2961265	Tarmon	Tarmon		52.53806	-9.37889	P	PPL	IE		M	11			0		76	Europe/Dublin	2010-08-14
+2961266	Tarbert House	Tarbert House		52.57833	-9.36833	S	EST	IE		M	11			0		1	Europe/Dublin	2010-08-14
+2961267	Tarbert	Tarbert	Tairbeart,Tarbert	52.57278	-9.37528	P	PPL	IE	IE	M	11			0		12	Europe/Dublin	2010-08-14
+2961268	Tara Hill	Tara Hill		52.69806	-6.21472	T	HLL	IE		L	30			0		56	Europe/Dublin	2010-08-14
+2961269	Tara Hall	Tara Hall		53.58	-6.60222	S	EST	IE		L	21			0		121	Europe/Dublin	2010-08-14
+2961270	Hill of Tara	Hill of Tara		53.57972	-6.61194	T	HLL	IE		L	21			0		135	Europe/Dublin	2010-08-14
+2961271	River Tar	River Tar		52.28333	-7.83333	H	STM	IE		00				0		48	Europe/Dublin	1993-12-27
+2961272	Lough Tap	Lough Tap		53.9	-7.98333	H	LK	IE		C	14			0		92	Europe/Dublin	1993-12-27
+2961273	Tanrego House	Tanrego House	Tanrego,Tanrego House	54.23056	-8.61222	S	HSEC	IE	IE	C	25			0		1	Europe/Dublin	2010-08-14
+2961274	Tankersley House	Tankersley House		52.82667	-6.38583	S	EST	IE		L	31			0		87	Europe/Dublin	2010-08-14
+2961275	Tang River	Tang River		53.53333	-7.83333	H	STM	IE		L	29			0		67	Europe/Dublin	1993-12-27
+2961276	Tang	Tang		53.53333	-7.78333	P	PPL	IE		L	29			0		67	Europe/Dublin	1993-12-27
+2961277	Lough Talt	Lough Talt		54.08389	-8.92139	H	LK	IE		C	25			0		182	Europe/Dublin	2010-08-14
+2961278	Tallyho Lodge	Tallyho Lodge		53.27	-8.69611	S	HSEC	IE		C	10			0		46	Europe/Dublin	2010-08-14
+2961279	Tallyho	Tallyho		52.81361	-6.36222	P	PPL	IE		L	31			0		170	Europe/Dublin	2010-08-14
+2961280	Tallow Road Station	Tallow Road Station	Tallow Road,Tallow Road Station	52.11667	-7.98333	S	RSTN	IE	IE	M	27			0		71	Europe/Dublin	2010-08-14
+2961281	Tallowbridge	Tallowbridge		52.10417	-8.00278	P	PPL	IE		M	27			0		32	Europe/Dublin	2010-08-14
+2961282	Tallow	Tallow	Tallow,Tulach an Iarainn	52.09278	-8.00806	P	PPL	IE		M	27			0		44	Europe/Dublin	2010-08-14
+2961283	Tallanstown	Tallanstown		53.92278	-6.54639	P	PPL	IE		L	19			0		36	Europe/Dublin	2010-08-14
+2961284	Tallaght	Tallaght	Tallaght	53.2859	-6.37344	P	PPL	IE	IE	L	07			64282		87	Europe/Dublin	2010-08-14
+2961285	Talbotstown House	Talbotstown House		53.22917	-6.45167	S	EST	IE		L	31			0		298	Europe/Dublin	2010-08-14
+2961286	Talbot Hall	Talbot Hall		52.38611	-6.9125	S	EST	IE		L	13			0		76	Europe/Dublin	2010-08-14
+2961287	Tahilla	Tahilla		51.83722	-9.83639	P	PPL	IE		M	11			0		62	Europe/Dublin	2010-08-14
+2961288	Tagoat	Tagoat		52.24278	-6.38028	P	PPL	IE		L	30			0		1	Europe/Dublin	2010-08-14
+2961289	Taghshinny	Taghshinny	Taghshinny	53.6	-7.71667	P	PPL	IE		L	18			0		76	Europe/Dublin	2010-08-10
+2961290	Taghsheenod	Taghsheenod		53.61667	-7.71667	P	PPL	IE		L	18			0		90	Europe/Dublin	1993-12-27
+2961291	Taghmon Castle	Taghmon Castle		53.6	-7.26667	S	BLDG	IE		L	29			0		118	Europe/Dublin	1993-12-27
+2961292	Taghmon	Taghmon	Taghmon,Teach Munna	52.32167	-6.6475	P	PPL	IE		L	30			0		73	Europe/Dublin	2010-08-14
+2961293	Tacumshin Lake	Tacumshin Lake	Tacumshin Lake,Tacumshin Lough,Tucumshin Lake	52.19528	-6.45306	H	BAY	IE	IE	L	30			0		-9999	Europe/Dublin	2010-08-14
+2961294	Lough Tacker	Lough Tacker		54.01667	-6.95	H	LK	IE		U	02			0		127	Europe/Dublin	2010-08-14
+2961295	Table Mountain	Table Mountain		53.01444	-6.48417	T	MT	IE		L	31			0	701	599	Europe/Dublin	2010-08-14
+2961296	Sybil Point	Sybil Point		52.17806	-10.47333	T	PT	IE		M	11			0		-9999	Europe/Dublin	2010-08-14
+2961297	Swords	Swords	Sord,Swords	53.45972	-6.21806	P	PPL	IE		L	35			29816		6	Europe/Dublin	2010-11-04
+2961298	Swines Head	Swines Head		52.12972	-7.02667	T	CAPE	IE		M	27			0		-9999	Europe/Dublin	2010-08-14
+2961299	Swinford	Swinford	Beal Atha na Muice,B�al �tha na Muice,Swineford,Swinford	53.95	-8.95	P	PPL	IE		C	20			1241		68	Europe/Dublin	2010-08-14
+2961300	River Swilly	River Swilly		54.96056	-7.67778	H	STM	IE		U	06			0		-9999	Europe/Dublin	2010-08-14
+2961301	Lough Swilly	Lough Swilly	Loch Suili,Loch S�il�,Lough Swilly,The Lake of Shadows	55.16667	-7.53333	H	INLT	IE		U	06			0		-9999	Europe/Dublin	2011-03-06
+2961302	The Sweep	The Sweep		52.21667	-7.23333	P	PPL	IE		M	27			0		49	Europe/Dublin	2010-08-14
+2961303	Swanlinbar	Swanlinbar	An Muileann Iarainn,Swanlinbar	54.19472	-7.70444	P	PPL	IE		U	02			0		74	Europe/Dublin	2010-08-14
+2961304	Swan	Swan		52.8875	-7.15972	P	PPL	IE		L	15			0		153	Europe/Dublin	2010-08-14
+2961305	Sutton	Sutton	Sutton	53.39056	-6.12167	P	PPL	IE		L	07			0		-9999	Europe/Dublin	2010-11-04
+2961306	Sunvale	Sunvale	Sun Ville,Sunvale	52.34611	-8.5175	S	EST	IE	IE	M	16			0		151	Europe/Dublin	2010-08-14
+2961307	Lough Sunderlin	Lough Sunderlin	Lough Sewdy,Lough Sunderlin	53.5	-7.66667	H	LK	IE		L	29			0		79	Europe/Dublin	2010-08-10
+2961308	Summerville	Summerville		53.8	-7.36667	P	PPL	IE		U	02			0		73	Europe/Dublin	1993-12-27
+2961309	Summerhill House	Summerhill House		53.47	-6.72556	S	EST	IE		L	21			0		78	Europe/Dublin	2010-08-14
+2961310	Summerhill House	Summerhill House		52.76306	-7.94278	S	EST	IE		M	26			0		107	Europe/Dublin	2010-08-14
+2961311	Summerhill House	Summerhill House	Summerhill House,Summerville	52.36778	-7.73889	S	EST	IE	IE	M	26			0		71	Europe/Dublin	2010-11-04
+2961312	Summerhill	Summerhill		53.47694	-6.73639	P	PPL	IE		L	21			0		77	Europe/Dublin	2010-08-14
+2961313	Summerhill	Summerhill		52.80528	-7.13722	P	PPL	IE		L	13			0		227	Europe/Dublin	2010-08-14
+2961314	Summer Cove	Summer Cove	Summer Cove,Summor Cove	51.70333	-8.50222	P	PPL	IE	IE	M	04			0		-9999	Europe/Dublin	2010-08-14
+2961315	Sullane River	Sullane River		51.88333	-8.93333	H	STM	IE		M	04			0		85	Europe/Dublin	2010-08-14
+2961316	Suir Mount	Suir Mount		52.29611	-7.79222	S	EST	IE		M	27			0		59	Europe/Dublin	2010-08-14
+2961317	Suir Island	Suir Island		52.35	-7.7	T	ISL	IE		M	27			0		132	Europe/Dublin	2010-08-14
+2961318	Suircastle House	Suircastle House		52.46167	-7.98972	S	HSEC	IE		M	26			0		73	Europe/Dublin	2010-08-14
+2961319	River Suir	River Suir	River Suir	52.27333	-6.99556	H	STM	IE		00				0		1	Europe/Dublin	2011-01-04
+2961320	Sugarloaf Mountain	Sugarloaf Mountain	Sugarloaf Mountain	51.72583	-9.63111	T	MT	IE		M	04			0		228	Europe/Dublin	2010-08-14
+2961321	Sugar Hill	Sugar Hill		52.43611	-9.16917	T	HLL	IE		M	11			0		312	Europe/Dublin	2010-08-14
+2961322	River Suck	River Suck		53.27222	-8.05306	H	STM	IE		00				0		55	Europe/Dublin	1998-08-18
+2961323	Sturrall	Sturrall		54.73667	-8.74417	T	CAPE	IE		U	06			0		-9999	Europe/Dublin	2010-08-14
+2961324	Stuake	Stuake		51.98333	-8.76667	P	PPL	IE		M	04			0		171	Europe/Dublin	2010-08-14
+2961325	Stroove	Stroove		55.21667	-6.93333	P	PPL	IE		U	06			0		-9999	Europe/Dublin	1993-12-27
+2961326	Strokestown House	Strokestown House		53.76667	-8.1	S	EST	IE		C	24			0		71	Europe/Dublin	1993-12-27
+2961327	Strokestown	Strokestown	Beal na mBuilli,B�al na mBuill�,Strokestown	53.78333	-8.1	P	PPL	IE		C	24			0		72	Europe/Dublin	2010-08-14
+2961328	Street	Street		53.66667	-7.48333	P	PPL	IE		L	29			0		72	Europe/Dublin	1993-12-27
+2961329	Streek Head	Streek Head		51.47056	-9.70306	T	CAPE	IE		M	04			0		-9999	Europe/Dublin	2010-08-14
+2961330	Streedagh Point	Streedagh Point		54.41194	-8.56722	T	PT	IE		C	25			0		1	Europe/Dublin	2010-08-14
+2961331	Streedagh House	Streedagh House		54.39778	-8.56389	S	HSEC	IE		C	25			0		1	Europe/Dublin	2010-08-14
+2961332	Streamstown Bay	Streamstown Bay		53.50917	-10.06833	H	BAY	IE		C	10			0		-9999	Europe/Dublin	2010-08-14
+2961333	Streamstown	Streamstown		53.52333	-10.04694	P	PPL	IE		C	10			0		76	Europe/Dublin	2010-08-14
+2961334	Streamstown	Streamstown		53.43333	-7.58333	P	PPL	IE		L	29			0		102	Europe/Dublin	1993-12-27
+2961335	Strawberryhill House	Strawberryhill House		53.23639	-7.88778	S	HSEC	IE		L	23			0		68	Europe/Dublin	2010-08-14
+2961336	Stratford	Stratford	Ath na Sraide,Stratford,�th na Sr�ide	52.98583	-6.67778	P	PPL	IE		L	31			0		147	Europe/Dublin	2010-08-14
+2961337	Stranorlar	Stranorlar	Srath an Urlair,Srath an Url�ir,Stranorlar	54.8	-7.76667	P	PPL	IE		U	06			0		58	Europe/Dublin	2010-08-14
+2961338	Strand	Strand		52.39556	-9.11056	P	PPL	IE		M	03			0		152	Europe/Dublin	2010-08-14
+2961339	Strancally House	Strancally House		52.01667	-7.85	S	BLDG	IE		M	04			0		71	Europe/Dublin	2010-08-14
+2961340	Strancally Castle	Strancally Castle		52.06861	-7.87694	S	EST	IE		M	27			0		36	Europe/Dublin	2010-08-14
+2961341	Straid River	Straid River		55.28333	-7.33333	H	STM	IE		U	06			0		40	Europe/Dublin	1993-12-27
+2961342	Straffan	Straffan		53.31556	-6.63472	P	PPL	IE		L	12			0		72	Europe/Dublin	2010-08-14
+2961343	Stradone House	Stradone House		53.98389	-7.25	S	EST	IE		U	02			0		134	Europe/Dublin	2010-08-14
+2961344	Stradone	Stradone	Sraith an Domhain,Stradone	53.98333	-7.23333	P	PPL	IE	IE	U	02			0		135	Europe/Dublin	2010-08-14
+2961345	Strade	Strade		53.91694	-9.13389	P	PPL	IE		C	20			0		38	Europe/Dublin	2010-08-14
+2961346	Stradbally River	Stradbally River	Bauteogue River,Stradbally River,Strade River,Straid River	53.03806	-7.08972	H	STM	IE	IE	L	15			0		72	Europe/Dublin	2010-08-14
+2961347	Stradbally Hall	Stradbally Hall		53	-7.16667	S	EST	IE		L	15			0		152	Europe/Dublin	2010-08-14
+2961348	Stradbally	Stradbally		53.21889	-8.88667	P	PPL	IE		C	10			0		2	Europe/Dublin	2010-08-14
+2961349	Stradbally	Stradbally	An Sraidbhaile,An Sr�idbhaile,Stradbally	53.01556	-7.15278	P	PPL	IE		L	15			0		149	Europe/Dublin	2010-08-14
+2961350	Stradbally	Stradbally		52.24722	-10.06639	P	PPL	IE		M	11			0		81	Europe/Dublin	2010-08-14
+2961351	Stradbally	Stradbally	An tSraidbhaile,An tSr�idbhaile,Stradbally	52.13	-7.46	P	PPL	IE		M	27			0		17	Europe/Dublin	2010-08-14
+2961352	Stowlin House	Stowlin House		53.18139	-8.21722	S	EST	IE		C	10			0		75	Europe/Dublin	2010-08-14
+2961353	Stookaruddan	Stookaruddan	Stockaruddan,Stookaruddan,Stookarudden	55.36972	-7.28222	T	ISLS	IE	IE	U	06			0		-9999	Europe/Dublin	2010-08-14
+2961354	Stonyford	Stonyford		52.53639	-7.22	P	PPL	IE		L	13			0		69	Europe/Dublin	2010-08-14
+2961355	Stonybatter	Stonybatter		52.76278	-6.45278	P	PPL	IE		L	31			0		159	Europe/Dublin	2010-08-14
+2961356	Stonetown	Stonetown		53.61667	-8.55	P	PPL	IE		C	10			0		76	Europe/Dublin	1993-12-27
+2961357	Stonepark	Stonepark		54.01028	-8.93722	P	PPL	IE		C	20			0		77	Europe/Dublin	2010-08-14
+2961358	Stonehall	Stonehall		52.60722	-8.87306	P	PPL	IE		M	16			0		22	Europe/Dublin	2010-08-14
+2961359	Stonefield	Stonefield		54.31	-9.82972	P	PPL	IE		C	20			0		141	Europe/Dublin	2010-08-14
+2961360	Stone Bridge	Stone Bridge		54.2	-7.16667	P	PPL	IE		U	22			0		73	Europe/Dublin	2010-08-14
+2961361	Stokestown	Stokestown	Stokestown,Stokestown Castle	52.36028	-6.98806	S	EST	IE	IE	L	30			0		13	Europe/Dublin	2010-08-14
+2961362	Stillorgan	Stillorgan		53.29049	-6.19487	P	PPL	IE		L	07			0		39	Europe/Dublin	2010-08-14
+2961363	Stephenstown House	Stephenstown House	Stephenstown,Stephenstown House	53.95806	-6.46639	S	EST	IE	IE	L	19			0		19	Europe/Dublin	2010-08-14
+2961364	Stepaside	Stepaside	Stepaside	53.2525	-6.21361	P	PPL	IE		L	07			0		140	Europe/Dublin	2010-11-04
+2961365	Station Island	Station Island		54.60833	-7.87139	T	ISL	IE		U	06			0		148	Europe/Dublin	2010-08-14
+2961366	Stamullin	Stamullin		53.62889	-6.26833	P	PPL	IE		L	21			0		48	Europe/Dublin	2010-08-14
+2961367	Stags of Broad Haven	Stags of Broad Haven	Stags,Stags of Broad Haven,The Stags	54.36639	-9.78778	T	RKS	IE	IE	C	20			0		-9999	Europe/Dublin	2010-08-14
+2961368	Stags of Bofin	Stags of Bofin		53.63194	-10.25917	T	RKS	IE		C	10			0		-9999	Europe/Dublin	2010-08-14
+2961369	The Stag Rocks	The Stag Rocks	The Stag Rocks,The Stags	55.07528	-8.47889	T	RKS	IE	IE	U	06			0		-9999	Europe/Dublin	2010-08-14
+2961370	Stacks Mountains	Stacks Mountains		52.31667	-9.56667	T	MTS	IE		M	11			0		286	Europe/Dublin	1993-12-27
+2961371	Stabannan	Stabannan	Stabannan,Stahannan	53.86472	-6.43583	P	PPL	IE	IE	L	19			0		6	Europe/Dublin	2010-08-14
+2961372	Sruwaddacon Bay	Sruwaddacon Bay	Sruwaddacon Bay,Sruwaddacon Creek	54.26667	-9.78333	H	BAY	IE		C	20			0		-9999	Europe/Dublin	2010-08-10
+2961373	Srahmore River	Srahmore River		53.95667	-9.58	H	STM	IE		C	20			0		98	Europe/Dublin	2010-08-14
+2961374	Srahmore	Srahmore		53.96028	-9.57083	P	PPL	IE		C	20			0		131	Europe/Dublin	2010-08-14
+2961375	Srahlaghy	Srahlaghy		54.235	-9.57361	P	PPL	IE		C	20			0		128	Europe/Dublin	2010-08-14
+2961376	Srahduggaun	Srahduggaun		54	-9.73333	P	PPL	IE		C	20			0		73	Europe/Dublin	1993-12-27
+2961377	Srah	Srah		54.1725	-9.93361	P	PPL	IE		C	20			0		5	Europe/Dublin	2010-08-14
+2961378	Srah	Srah		53.68333	-9.33333	P	PPL	IE		C	20			0		59	Europe/Dublin	1993-12-27
+2961379	Srah	Srah		53.28333	-7.75	P	PPL	IE		L	23			0		65	Europe/Dublin	2010-08-14
+2961380	Spring Valley	Spring Valley		53.47806	-6.71389	S	EST	IE		L	21			0		77	Europe/Dublin	2010-08-14
+2961381	Springfield Castle	Springfield Castle		52.35	-8.95	S	EST	IE		M	16			0		146	Europe/Dublin	1993-12-27
+2961382	Springfield	Springfield		53.05	-7.4	P	PPL	IE		L	15			0		138	Europe/Dublin	2010-08-14
+2961383	Springfield	Springfield		52.71556	-6.615	S	EST	IE		L	31			0		71	Europe/Dublin	2010-08-14
+2961384	Sporthouse Cross Roads	Sporthouse Cross Roads		52.19667	-7.17917	P	PPL	IE		M	27			0		77	Europe/Dublin	2010-08-14
+2961385	Spittle	Spittle		52.38	-8.3375	P	PPL	IE		M	16			0		152	Europe/Dublin	2010-08-14
+2961386	Spink	Spink		52.89944	-7.22611	P	PPL	IE		L	15			0		151	Europe/Dublin	2010-08-14
+2961387	Spike Island	Spike Island		51.83861	-8.28889	T	ISL	IE		M	04			0		-9999	Europe/Dublin	2010-08-14
+2961388	Spiddal	Spiddal	An Spideal,An Spid�al,Spiddal,Spiddle	53.24667	-9.30278	P	PPL	IE		C	10			0		1	Europe/Dublin	2010-08-14
+2961389	Spear Vale	Spear Vale		53.91667	-7	P	PPL	IE		U	02			0		187	Europe/Dublin	1993-12-27
+2961390	Spanish Point	Spanish Point		52.84972	-9.44722	T	PT	IE		M	03			0		-9999	Europe/Dublin	2010-08-14
+2961391	Spancelhill	Spancelhill		52.86667	-8.9	P	PPL	IE		M	03			0		119	Europe/Dublin	1993-12-27
+2961392	Spa	Spa		52.28333	-9.78333	P	PPL	IE		M	11			0		39	Europe/Dublin	2010-08-14
+2961393	River Sow	River Sow		52.37778	-6.45139	H	STM	IE		L	30			0		4	Europe/Dublin	2010-08-14
+2961394	Sovereign Islands	Sovereign Islands	Big Sovereign,Sovereign Islands	51.67667	-8.45556	T	ISLS	IE	IE	M	04			0		-9999	Europe/Dublin	2010-08-14
+2961395	Sovereign Island	Sovereign Island	Little Sovereign,Sovereign Island	51.68389	-8.4425	T	ISL	IE	IE	M	04			0		-9999	Europe/Dublin	2010-08-14
+2961396	South Sound	South Sound		53.04167	-9.46583	H	SD	IE		M	03			0		-9999	Europe/Dublin	2010-08-14
+2961397	Southpark House	Southpark House		53.76667	-8.43333	S	EST	IE		C	24			0		77	Europe/Dublin	1993-12-27
+2961398	South Hill	South Hill		53.60861	-7.07167	S	EST	IE		L	29			0		109	Europe/Dublin	2010-08-14
+2961399	Sorrel House	Sorrel House		52.75	-9.23333	S	BLDG	IE		M	03			0		96	Europe/Dublin	1993-12-27
+2961400	Sorrelhill House	Sorrelhill House		52.83333	-7.75	S	EST	IE		M	26			0		150	Europe/Dublin	2010-08-14
+2961401	Sopwell Hall	Sopwell Hall		52.99194	-8.04806	S	EST	IE		M	26			0		119	Europe/Dublin	2010-08-14
+2961402	Soppog	Soppog		55.06472	-7.32111	P	PPL	IE		U	06			0		63	Europe/Dublin	2010-08-14
+2961403	Sonna House	Sonna House		53.56667	-7.46667	S	EST	IE		L	29			0		123	Europe/Dublin	1993-12-27
+2961404	Somerville House	Somerville House	Somerville,Somerville House	53.63139	-6.51167	S	EST	IE	IE	L	21			0		57	Europe/Dublin	2010-08-14
+2961405	Somerset House	Somerset House		53.24778	-8.22694	S	EST	IE		C	10			0		76	Europe/Dublin	2010-08-14
+2961406	Solsborough House	Solsborough House	Solsborough,Solsborough House	52.52861	-6.52	S	EST	IE	IE	L	30			0		64	Europe/Dublin	2010-08-14
+2961407	Snowhill House	Snowhill House		52.27278	-7.02028	S	EST	IE		L	13			0		1	Europe/Dublin	2010-08-14
+2961408	Sneem Harbour	Sneem Harbour		51.8	-9.88333	H	INLT	IE		M	11			0		-9999	Europe/Dublin	2010-08-14
+2961409	Sneem	Sneem	An Snaidhm,An tSnaidhm,Sneem	51.83333	-9.9	P	PPL	IE		M	11			0		25	Europe/Dublin	2010-08-14
+2961410	Slieve Snaght	Slieve Snaght	Sliabh Sneachta,Slieve Snaght	55.19639	-7.33528	T	MT	IE		U	06			0	615	573	Europe/Dublin	2010-08-14
+2961411	Slieve Snaght	Slieve Snaght		54.98083	-8.11778	T	MT	IE		U	06			0		466	Europe/Dublin	2010-08-14
+2961412	Smithstown House	Smithstown House		52.98333	-9.26667	S	EST	IE		M	03			0		75	Europe/Dublin	1993-12-27
+2961413	Smithborough	Smithborough		54.22333	-7.09083	P	PPL	IE		U	22			0		74	Europe/Dublin	2010-08-14
+2961414	Smerwick Harbour	Smerwick Harbour		52.19222	-10.4025	H	HBR	IE		M	11			0		-9999	Europe/Dublin	2010-08-14
+2961415	Smerwick	Smerwick		52.19444	-10.42333	P	PPL	IE		M	11			0		30	Europe/Dublin	2010-08-14
+2961416	Smearlagh River	Smearlagh River		52.43917	-9.42889	H	STM	IE		M	11			0		74	Europe/Dublin	2010-08-14
+2961417	Smarmore Castle	Smarmore Castle		53.81722	-6.56667	S	EST	IE		L	19			0		75	Europe/Dublin	2010-08-14
+2961418	Slyne Head	Slyne Head		53.40056	-10.23528	T	CAPE	IE		C	10			0		-9999	Europe/Dublin	2010-08-14
+2961419	Slugga	Slugga		54.30278	-9.86556	T	RK	IE		C	20			0		-9999	Europe/Dublin	2010-08-14
+2961420	Slish Wood	Slish Wood		54.23333	-8.38333	V	FRST	IE		C	25			0		149	Europe/Dublin	2010-08-14
+2961421	Sligo Bay	Sligo Bay		54.3	-8.7	H	BAY	IE		C	25			0		-9999	Europe/Dublin	1998-09-01
+2961422	Sligeach	Sligeach	Contae Shligigh,County Sligo,Sligo	54.25	-8.66667	A	ADM2	IE		C	25			58900		5	Europe/Dublin	2010-08-14
+2961423	Sligeach	Sligeach	Slajgou,Sligeach,Sligo,suraigo,\u0421\u043b\u0430\u0439\u0433\u043e\u0443,\u30b9\u30e9\u30a4\u30b4	54.26969	-8.46943	P	PPLA2	IE		C	25			20228	13	5	Europe/Dublin	2010-08-14
+2961424	Slievetooey	Slievetooey		54.76278	-8.60722	T	MT	IE		U	06			0		329	Europe/Dublin	2010-08-14
+2961425	Slieveroe	Slieveroe		52.28056	-7.08667	P	PPL	IE		L	13			0		59	Europe/Dublin	2010-08-14
+2961426	Slievenamuck	Slievenamuck		52.4275	-8.23194	T	MT	IE		M	26			0		156	Europe/Dublin	2010-08-14
+2961427	Slievenamon	Slievenamon	Slievenaman,Slievenamon	52.42778	-7.56111	T	MT	IE		M	26			0	721	568	Europe/Dublin	2010-11-04
+2961428	Slievenakilla	Slievenakilla		54.15	-7.93333	P	PPL	IE		C	14			0		315	Europe/Dublin	1993-12-27
+2961429	Slievemore Point	Slievemore Point		54.01667	-10.05	T	PT	IE		C	20			0		73	Europe/Dublin	1993-12-27
+2961430	Slievemore	Slievemore		53.99611	-10.07056	P	PPLL	IE		C	20			0		149	Europe/Dublin	2010-08-14
+2961431	Slievemore	Slievemore		54.0075	-10.0625	T	MT	IE		C	20			0	671	269	Europe/Dublin	2010-08-14
+2961432	Slieve Miskish Mountains	Slieve Miskish Mountains	Slieve Miskish Mountains	51.66722	-9.95306	T	MTS	IE		M	04			0		188	Europe/Dublin	2010-08-14
+2961433	Slieve Mish Mountains	Slieve Mish Mountains	Slieve Mish Mountains	52.2	-9.76667	T	MTS	IE		M	11			0		224	Europe/Dublin	2010-08-14
+2961434	Slievekimalta	Slievekimalta	Keeper Hill,Slievekimalta	52.74972	-8.26028	T	MT	IE	IE	M	26			0		421	Europe/Dublin	2010-11-04
+2961435	Slieveglass	Slieveglass		52.27861	-10.20667	T	MT	IE		M	11			0		143	Europe/Dublin	2010-08-14
+2961436	Slievefelim Mountains	Slievefelim Mountains		52.67167	-8.3075	T	MTS	IE		M	16			0		315	Europe/Dublin	2010-08-14
+2961437	Slievecarran	Slievecarran		53.09583	-9.00639	T	MT	IE		M	03			0		154	Europe/Dublin	2010-08-14
+2961438	Slievecallan	Slievecallan		52.84056	-9.26861	T	MT	IE		M	03			0		279	Europe/Dublin	2010-08-14
+2961439	Slieveboy	Slieveboy		52.65639	-6.48861	T	MT	IE		L	30			0	422	266	Europe/Dublin	2010-08-14
+2961440	Slieve Bloom Mountains	Slieve Bloom Mountains	Slieve Bloom,Slieve Bloom Mountains	53.09333	-7.56722	T	MTS	IE		L	15			0		326	Europe/Dublin	2010-08-10
+2961441	Slievebane Bay	Slievebane Bay		55.36667	-7.33333	H	BAY	IE		U	06			0		1	Europe/Dublin	2010-08-14
+2961442	Slieve Aughty Mountains	Slieve Aughty Mountains	Slieve Aughty Mountains	53.04611	-8.51111	T	MTS	IE		C	10			0		151	Europe/Dublin	2010-08-14
+2961443	Slieveardagh Hills	Slieveardagh Hills	Slievardagh Region,Slieveardagh Hills	52.65278	-7.52056	T	HLLS	IE	IE	00				0		152	Europe/Dublin	2010-08-10
+2961444	Slevoir House	Slevoir House		53.06139	-8.18111	S	EST	IE		M	26			0		56	Europe/Dublin	2010-08-14
+2961445	Sleatygraigue	Sleatygraigue		52.84472	-6.97167	P	PPL	IE		L	15			0		39	Europe/Dublin	2010-08-14
+2961446	Slea Head	Slea Head		52.09639	-10.45917	T	CAPE	IE		M	11			0		-9999	Europe/Dublin	2010-08-14
+2961447	Slate River	Slate River		53.195	-7.09417	H	STM	IE		L	23			0		75	Europe/Dublin	1999-02-24
+2961448	River Slaney	River Slaney		52.33806	-6.45306	H	STM	IE		L	30			0		-9999	Europe/Dublin	2010-08-14
+2961449	Slane Castle	Slane Castle		53.70944	-6.5825	S	EST	IE		L	21			0		37	Europe/Dublin	2010-08-14
+2961450	Slane	Slane	Baile Shlaine,Baile Shl�ine,Slane	53.71	-6.54333	P	PPL	IE		L	21			0		58	Europe/Dublin	2010-08-14
+2961451	Slaheny River	Slaheny River		51.9	-9.45	H	STM	IE		M	11			0		86	Europe/Dublin	1993-12-27
+2961452	Slade	Slade		52.1325	-6.90444	P	PPL	IE		L	30			0		-9999	Europe/Dublin	2010-08-14
+2961453	Skycur House	Skycur House	Skycur,Skycur House	53.22139	-8.23472	S	EST	IE	IE	C	10			0		76	Europe/Dublin	2010-08-14
+2961454	Skull Harbour	Skull Harbour		51.52528	-9.53833	H	HBR	IE		M	04			0		-9999	Europe/Dublin	2010-08-14
+2961455	Skull	Skull	An Scoil,Schull,Skull	51.53333	-9.53333	P	PPL	IE		M	04			0		44	Europe/Dublin	2010-08-14
+2961456	Skreeny House	Skreeny House		54.315	-8.16722	S	RUIN	IE		C	14			0		76	Europe/Dublin	2010-08-14
+2961457	Skreen	Skreen		54.24333	-8.73111	P	PPL	IE		C	25			0		55	Europe/Dublin	2010-08-14
+2961458	Screen	Screen	Screen,Skreen	52.41361	-6.4125	P	PPL	IE	IE	L	30			0		22	Europe/Dublin	2010-08-14
+2961459	An Sciobairin	An Sciobairin	Skibbereen	51.55	-9.26667	P	PPL	IE		M	04			2098		98	Europe/Dublin	2010-08-14
+2961460	Skerries Islands	Skerries Islands		53.58333	-6.08333	T	ISLS	IE		00				0		-9999	Europe/Dublin	1993-12-27
+2961461	Skerries	Skerries	Na Sceiri,Na Sceir�,Skerries	53.58278	-6.10833	P	PPL	IE		L	35			10014		-9999	Europe/Dublin	2010-11-04
+2961462	Skenakilla Cross Roads	Skenakilla Cross Roads		52.19306	-8.51778	P	PPL	IE		M	04			0		76	Europe/Dublin	2010-08-14
+2961463	Skellig Rocks	Skellig Rocks	Skellig Rocks,The Skelligs	51.76667	-10.51667	T	RKS	IE	IE	M	11			0		-9999	Europe/Dublin	2010-08-14
+2961464	Skehanagh	Skehanagh		53.41389	-8.63417	P	PPL	IE		C	10			0		72	Europe/Dublin	2010-08-14
+2961465	Lough Skean	Lough Skean		54.06222	-8.21528	H	LK	IE		00				0		55	Europe/Dublin	1998-09-01
+2961466	Skeagh Lough Upper	Skeagh Lough Upper	Skeagh Lough,Skeagh Lough Upper	53.95	-7	H	LK	IE	IE	U	02			0		150	Europe/Dublin	2010-08-14
+2961467	Skeagh	Skeagh		51.58639	-9.35472	P	PPL	IE		M	04			0		79	Europe/Dublin	2010-08-14
+2961468	Skeaf House	Skeaf House		51.6775	-8.78111	S	EST	IE		M	04			0		75	Europe/Dublin	2010-08-14
+2961469	Lough Skannive	Lough Skannive		53.33083	-9.78722	H	LK	IE		C	10			0		65	Europe/Dublin	2010-08-14
+2961470	Sixmilebridge	Sixmilebridge	Droichead Abhann O gCearnaigh,Dro�chead Abhann � gCearnaigh,Sixmilebridge	52.74139	-8.77417	P	PPL	IE		M	03			0		43	Europe/Dublin	2010-08-14
+2961471	Single Street	Single Street		54.48306	-8.26417	P	PPL	IE		U	06			0		1	Europe/Dublin	2010-08-14
+2961472	Silver Stream	Silver Stream		54.26667	-6.91667	P	PPL	IE		00				0		66	Europe/Dublin	1993-12-27
+2961473	Silver River	Silver River		53.2925	-7.62889	H	STM	IE		L	23			0		70	Europe/Dublin	2010-08-14
+2961474	Silver River	Silver River		53.24333	-7.79139	H	STM	IE		L	23			0		63	Europe/Dublin	2010-08-14
+2961475	Silvermines	Silvermines	Beal Atha Gabhann,B�al �tha Gabhann,Silvermines	52.79028	-8.23417	P	PPL	IE		M	38			0		188	Europe/Dublin	2010-11-04
+2961476	Silvermine Mountains	Silvermine Mountains	Silvermine Mountains,Silvermines Mountains	52.77583	-8.24889	T	MTS	IE	IE	M	26			0		345	Europe/Dublin	2010-11-04
+2961477	Silver Hill	Silver Hill		54.76944	-8.14722	T	MT	IE		U	06			0		423	Europe/Dublin	2010-08-14
+2961478	Lough Sillan	Lough Sillan		54.00222	-6.92222	H	LK	IE		U	02			0		149	Europe/Dublin	2010-08-14
+2961479	Siddan	Siddan		53.80972	-6.65639	P	PPL	IE		L	21			0		61	Europe/Dublin	2010-08-14
+2961480	Shrule Castle	Shrule Castle		52.87694	-6.95111	S	BLDG	IE		L	15			0		48	Europe/Dublin	2010-08-14
+2961481	Shrule	Shrule	Shrule,Shruthair,Sruthair	53.51681	-9.08902	P	PPL	IE		C	20			0		59	Europe/Dublin	2010-08-14
+2961482	Shronowen	Shronowen		52.50389	-9.45361	P	PPL	IE		M	11			0		69	Europe/Dublin	2010-08-14
+2961483	Shronell	Shronell		52.47222	-8.23472	P	PPL	IE		M	26			0		136	Europe/Dublin	2010-08-14
+2961484	Shrone	Shrone		52.525	-9.49	P	PPL	IE		M	11			0		57	Europe/Dublin	2010-08-14
+2961485	Shot Head	Shot Head		51.67667	-9.67611	T	CAPE	IE		M	04			0		-9999	Europe/Dublin	2010-08-14
+2961486	Shiven River	Shiven River	Kingstown River,Shiven River	53.51667	-8.45	H	STM	IE		C	10			0		72	Europe/Dublin	2010-08-10
+2961487	Shiven River	Shiven River		53.5	-8.28333	H	STM	IE		00				0		69	Europe/Dublin	1993-12-27
+2961488	Shippool	Shippool	Shippool,Shippool Castle	51.75111	-8.62889	S	RUIN	IE	IE	M	04			0		72	Europe/Dublin	2010-08-14
+2961489	Shinrone	Shinrone	Shinrone,Sui an Roin,Su� an R�in	52.9825	-7.92444	P	PPL	IE	IE	L	23			0		84	Europe/Dublin	2010-08-14
+2961490	Lough Shindilla	Lough Shindilla		53.45	-9.56667	H	LK	IE		C	10			0		58	Europe/Dublin	1993-12-27
+2961491	Shillelagh	Shillelagh	Shillelagh,Siol Ealaigh,S�ol �alaigh	52.75389	-6.53722	P	PPL	IE	IE	L	31			0		76	Europe/Dublin	2010-08-14
+2961492	Sheskin Lodge	Sheskin Lodge		54.17028	-9.615	S	HSE	IE		C	20			0		146	Europe/Dublin	2010-08-14
+2961493	Sheshia	Sheshia		53.1	-9.06667	P	PPL	IE		M	03			0		57	Europe/Dublin	1993-12-27
+2961494	Sherky Island	Sherky Island		51.7975	-9.905	T	ISL	IE		M	11			0		-9999	Europe/Dublin	2010-08-14
+2961495	Sherkin Island	Sherkin Island	Sherkin Island	51.4775	-9.42694	T	ISL	IE		M	04			0		1	Europe/Dublin	2010-08-14
+2961496	Sheriffhill	Sheriffhill		52.95	-6.80278	P	PPL	IE		L	12			0		143	Europe/Dublin	2010-08-14
+2961497	Shercock	Shercock	Searcog,Searc�g,Shercock	54	-6.9	P	PPL	IE	IE	U	02			0		153	Europe/Dublin	2010-08-14
+2961498	Shenicks Island	Shenicks Island		53.57472	-6.08444	T	ISL	IE		L	07			0		-9999	Europe/Dublin	2010-08-14
+2961499	Shelton Abbey	Shelton Abbey		52.815	-6.18278	S	EST	IE		L	31			0		43	Europe/Dublin	2010-08-14
+2961500	Shehy Mountain	Shehy Mountain		51.81778	-9.28722	T	MT	IE		M	04			0		334	Europe/Dublin	2010-08-14
+2961501	Sheffield	Sheffield	Sheffield,Sheffield House	53.00417	-7.26806	S	EST	IE	IE	L	15			0		150	Europe/Dublin	2010-08-14
+2961502	Sheep Haven	Sheep Haven	Cuan na gCaorach,Sheep Haven	55.2	-7.9	H	BAY	IE		U	06			0		-9999	Europe/Dublin	2010-08-14
+2961503	Sheen River	Sheen River		51.86667	-9.56667	H	STM	IE		M	11			0		61	Europe/Dublin	1993-12-27
+2961504	Lough Sheelin	Lough Sheelin		53.8	-7.31667	H	LK	IE		L	21			0		73	Europe/Dublin	1993-12-27
+2961505	Sheehills House	Sheehills House		52.93278	-7.76861	S	EST	IE		M	26			0		135	Europe/Dublin	2010-08-14
+2961506	Sheeanamore	Sheeanamore		52.885	-6.37694	P	PPL	IE		L	31			0		269	Europe/Dublin	2010-08-14
+2961507	Shark Head	Shark Head	Shark Head,Stark Head	53.60167	-10.295	T	CAPE	IE	IE	C	10			0		-9999	Europe/Dublin	2010-08-14
+2961508	Shanvally	Shanvally		53.78333	-7.91667	P	PPL	IE		C	24			0		67	Europe/Dublin	1993-12-27
+2961509	Shantonagh	Shantonagh		54.05028	-6.85056	P	PPL	IE		U	22			0		147	Europe/Dublin	2010-08-14
+2961510	Shanow River	Shanow River		52.36667	-9.65	H	STM	IE		M	11			0		70	Europe/Dublin	1993-12-27
+2961511	Shannon View	Shannon View		52.61667	-8.98333	S	EST	IE		M	16			0		3	Europe/Dublin	1993-12-27
+2961512	Shannon Harbour	Shannon Harbour	Shannon Harbour	53.22056	-7.94833	P	PPL	IE		L	23			0		73	Europe/Dublin	2010-08-14
+2961513	Shannon Hall	Shannon Hall	Shannon Hall,Shannonvale House	52.90417	-8.28056	S	BLDG	IE	IE	M	26			0		51	Europe/Dublin	2010-11-04
+2961514	Shannongrove House	Shannongrove House	Shannongrove,Shannongrove House	52.66	-8.86417	S	EST	IE	IE	M	16			0		1	Europe/Dublin	2010-08-14
+2961515	Shannon Grove	Shannon Grove		53.19861	-8.04806	S	EST	IE		C	10			0		55	Europe/Dublin	2010-08-14
+2961516	Shannonbridge	Shannonbridge	Droichead na Sionainne,Shannonbridge	53.27778	-8.03722	P	PPL	IE		L	23			0		56	Europe/Dublin	2010-08-14
+2961517	River Shannon	River Shannon		52.58194	-9.68167	H	STM	IE		00				0		-9999	Europe/Dublin	1998-05-05
+2961518	Mouth of the Shannon	Mouth of the Shannon		52.50167	-9.81694	H	STMM	IE		M	11			0		-9999	Europe/Dublin	2010-08-14
+2961519	Shannaghmore	Shannaghmore		53.6	-8.6	P	PPL	IE		C	10			0		76	Europe/Dublin	1993-12-27
+2961520	Shanlis Cross Roads	Shanlis Cross Roads	Shanlis,Shanlis Cross Roads	53.83778	-6.57111	P	PPL	IE	IE	L	19			0		49	Europe/Dublin	2010-08-14
+2961521	Shanlaragh	Shanlaragh		51.79556	-9.07889	P	PPL	IE		M	04			0		152	Europe/Dublin	2010-08-14
+2961522	Shankill Castle	Shankill Castle		52.68611	-7.02111	S	EST	IE		L	13			0		60	Europe/Dublin	2010-08-14
+2961523	Shankill	Shankill	Shankill	53.22611	-6.12444	P	PPL	IE		L	07			0		1	Europe/Dublin	2010-11-04
+2961524	Shanid Castle	Shanid Castle		52.55	-9.11667	S	BLDG	IE		M	16			0		117	Europe/Dublin	1993-12-27
+2961525	Shanganagh Junction	Shanganagh Junction		53.21667	-6.1	S	RSTN	IE		00				0		-9999	Europe/Dublin	1993-12-27
+2961526	Shanganagh Castle	Shanganagh Castle		53.22583	-6.11972	S	EST	IE		L	07			0		1	Europe/Dublin	2010-08-14
+2961527	Shanballymore	Shanballymore	An Seanbhaile Mor,An Seanbhaile M�r,Shanballymore	52.22028	-8.50139	P	PPL	IE		M	04			0		70	Europe/Dublin	2010-08-14
+2961528	Shanbally House	Shanbally House	Shanbally,Shanbally House	52.85722	-8.09167	S	EST	IE	IE	M	26			0		105	Europe/Dublin	2010-11-04
+2961529	Shanbally Castle	Shanbally Castle		52.29361	-8.04361	S	EST	IE		M	26			0		75	Europe/Dublin	2010-08-14
+2961530	Shanbally	Shanbally		51.8325	-8.3575	P	PPL	IE		M	04			0		11	Europe/Dublin	2010-08-14
+2961531	Shanagolden	Shanagolden	Seanghualainn,Shanagolden	52.5725	-9.1025	P	PPL	IE	IE	M	16			0		67	Europe/Dublin	2010-08-14
+2961532	Shanagh	Shanagh	Shanagh,Shunagh	51.58222	-8.71889	P	PPL	IE	IE	M	04			0		13	Europe/Dublin	2010-08-14
+2961533	Shanagarry	Shanagarry	Shanagarry	51.85444	-8.07083	P	PPL	IE		M	04			0		51	Europe/Dublin	2010-08-14
+2961534	Shallee River	Shallee River		52.87056	-8.99528	H	STM	IE		M	03			0		24	Europe/Dublin	2010-08-14
+2961535	Shaen House	Shaen House	Shaen,Shaen House	53.06667	-7.25	S	EST	IE	IE	L	15			0		141	Europe/Dublin	2010-08-14
+2961536	Shad Lough	Shad Lough		53.73333	-8.26667	H	LK	IE		C	24			0		76	Europe/Dublin	1993-12-27
+2961537	The Seven Stars	The Seven Stars		53.05722	-6.88028	L	LCTY	IE		L	12			0		108	Europe/Dublin	2010-08-14
+2961538	The Seven Hogs	The Seven Hogs	Magharee Islands,The Seven Hogs	52.32194	-10.03528	T	ISLS	IE	IE	M	11			0		-9999	Europe/Dublin	2010-08-14
+2961539	Seven Heads Bay	Seven Heads Bay		51.59444	-8.7	H	BAY	IE		M	04			0		-9999	Europe/Dublin	2010-08-14
+2961540	Seven Heads	Seven Heads		51.56889	-8.7275	T	CAPE	IE		M	04			0		-9999	Europe/Dublin	2010-08-14
+2961541	Seven Fathom Bank	Seven Fathom Bank		52.8	-6	H	BNK	IE		L	31			0		-9999	Europe/Dublin	1993-12-27
+2961542	Sevenchurches	Sevenchurches	Camaderry,Sevenchurches	53.02389	-6.39306	S	RUIN	IE	IE	L	31			0		588	Europe/Dublin	2010-08-14
+2961543	The Seven Arches	The Seven Arches	Seven Arches,The Seven Arches	55.21667	-7.6	S	CAVE	IE		U	06			0		-9999	Europe/Dublin	2010-08-10
+2961544	Seskinryan	Seskinryan		52.68222	-6.93167	P	PPL	IE		L	01			0		115	Europe/Dublin	2010-08-14
+2961545	Seskinrea	Seskinrea		52.76778	-7.06833	P	PPL	IE		L	01			0		191	Europe/Dublin	2010-08-14
+2961546	Seltannaveeny	Seltannaveeny	Seltannaveeny,Seltannavenny	54.1	-8.11667	P	PPL	IE		C	24			0		145	Europe/Dublin	2010-08-10
+2961547	Sellerna Bay	Sellerna Bay	Sellerna Bay,Sillerna Bay	53.55	-10.13333	H	BAY	IE		C	10			0		61	Europe/Dublin	2010-08-10
+2961548	Seershin	Seershin		53.26667	-9.21667	P	PPL	IE		C	10			0		57	Europe/Dublin	2010-08-14
+2961549	Seaweed Point	Seaweed Point		53.24889	-9.10944	T	PT	IE		C	10			0		-9999	Europe/Dublin	2010-08-14
+2961550	Seaview House	Seaview House	Seaview,Seaview House	52.02472	-7.58556	S	EST	IE	IE	M	27			0		2	Europe/Dublin	2010-08-14
+2961551	Seapark Point	Seapark Point		52.93333	-6.01667	T	PT	IE		L	31			0		-9999	Europe/Dublin	2010-08-14
+2961552	Sea Mount	Sea Mount		53.44278	-6.14	P	PPL	IE		L	07			0		1	Europe/Dublin	2010-08-14
+2961553	Seal Rocks	Seal Rocks		54.33667	-8.68722	T	RKS	IE		C	25			0		-9999	Europe/Dublin	2010-08-14
+2961554	Seafield House	Seafield House	Seafield,Seafield House	52.67417	-6.21444	S	EST	IE	IE	L	30			0		-9999	Europe/Dublin	2010-08-14
+2961555	Scurmore House	Scurmore House	Scarmore House,Scurmore House	54.19667	-9.11556	S	HSEC	IE	IE	C	25			0		3	Europe/Dublin	2010-08-14
+2961556	Lough Scur	Lough Scur	Lough Scur,Lough Seur	54.02667	-7.95222	H	LK	IE	IE	C	14			0		65	Europe/Dublin	2010-08-14
+2961557	Scullogue Gap	Scullogue Gap	Sculloge Gap,Scullogue Gap	52.57306	-6.7875	T	GAP	IE	IE	L	01			0		286	Europe/Dublin	2010-08-14
+2961558	Scullane Point	Scullane Point		51.49139	-9.21139	T	PT	IE		M	04			0		-9999	Europe/Dublin	2010-08-14
+2961559	Scregg West	Scregg West		53.53333	-8.58333	P	PPL	IE		C	10			0		76	Europe/Dublin	1993-12-27
+2961560	Scregg House	Scregg House		53.55	-8.1	S	EST	IE		C	24			0		76	Europe/Dublin	1993-12-27
+2961561	Scregg East	Scregg East		53.53333	-8.56667	P	PPL	IE		C	10			0		76	Europe/Dublin	1993-12-27
+2961562	Screeb Lodge	Screeb Lodge	Screeb,Screeb Lodge	53.38694	-9.5525	S	HSEC	IE	IE	C	10			0		69	Europe/Dublin	2010-08-14
+2961563	Scramore Loughs	Scramore Loughs		54.3	-8.35	H	LKS	IE		00				0		284	Europe/Dublin	1993-12-27
+2961564	Scramoge	Scramoge		53.76667	-8.05	P	PPL	IE		C	24			0		76	Europe/Dublin	1993-12-27
+2961565	Scrahan	Scrahan		52.4	-9.36667	P	PPL	IE		M	11			0		143	Europe/Dublin	1993-12-27
+2961566	Scrabby	Scrabby		53.86667	-7.53333	P	PPL	IE		U	02			0		70	Europe/Dublin	1993-12-27
+2961567	Scotstown	Scotstown	Baile an Scotaigh,Scotstown	54.2775	-7.065	P	PPL	IE		U	22			0		81	Europe/Dublin	2010-08-14
+2961568	Scotshouse	Scotshouse		54.12194	-7.2475	P	PPL	IE		U	22			0		78	Europe/Dublin	2010-08-14
+2961569	Scilly	Scilly		51.70639	-8.52167	P	PPL	IE		M	04			0		1	Europe/Dublin	2010-08-14
+2961570	Scattery Island	Scattery Island		52.61222	-9.51944	T	ISL	IE		M	03			0		-9999	Europe/Dublin	2010-08-14
+2961571	Scartaglin	Scartaglin	Scartaglen,Scartaglin,Seartaglin	52.18333	-9.4	P	PPL	IE		M	11			0		149	Europe/Dublin	2010-08-10
+2961572	Scarriff Bay	Scarriff Bay	Scariff Bay,Scarriff Bay	52.90333	-8.48694	H	BAY	IE	IE	M	03			0		55	Europe/Dublin	2010-08-14
+2961573	River Scarriff	River Scarriff		52.9	-8.5	H	STM	IE		M	03			0		55	Europe/Dublin	1993-12-27
+2961574	Scarriff	Scarriff	An Scairbh,Scariff,Scarriff	52.89889	-8.50917	P	PPL	IE		M	03			0		82	Europe/Dublin	2010-08-14
+2961575	Scariff Island	Scariff Island	Great Hog Island,Scariff Island	51.73556	-10.25333	T	ISL	IE	IE	M	11			0		47	Europe/Dublin	2010-08-14
+2961576	Scardaun	Scardaun		53.65	-9	P	PPL	IE		C	20			0		120	Europe/Dublin	1993-12-27
+2961577	Lough Scannive	Lough Scannive		53.43528	-9.9525	H	LK	IE		C	10			0		81	Europe/Dublin	2010-08-14
+2961578	Scalp Mountain	Scalp Mountain		55.08917	-7.36611	T	MT	IE		U	06			0	484	352	Europe/Dublin	2010-08-14
+2961579	The Scalp	The Scalp	An Scailp,The Scalp	53.21611	-6.17944	T	GAP	IE	IE	00				0		151	Europe/Dublin	2010-08-10
+2961580	Scalp	Scalp		53.08333	-6.58333	P	PPL	IE		L	31			0		304	Europe/Dublin	2010-08-14
+2961581	Scalp	Scalp		52.99139	-8.48972	T	MT	IE		C	10			0		304	Europe/Dublin	2010-08-14
+2961582	Saundersville	Saundersville		52.96667	-6.68333	P	PPL	IE		L	31			0		156	Europe/Dublin	1993-12-27
+2961583	Saunders Court	Saunders Court		52.36611	-6.49333	S	EST	IE		L	30			0		1	Europe/Dublin	2010-08-14
+2961584	Sarsfieldscourt House	Sarsfieldscourt House	Sarsfieldcourt,Sarsfieldscourt House	51.95139	-8.4025	S	HSEC	IE	IE	M	04			0		89	Europe/Dublin	2010-08-14
+2961585	Sarnaght	Sarnaght		53.89056	-9.31389	P	PPL	IE		C	20			0		104	Europe/Dublin	2010-08-14
+2961586	Sarahville	Sarahville		52.16667	-7.46667	S	EST	IE		M	27			0		78	Europe/Dublin	1993-12-27
+2961587	Santry	Santry		53.39812	-6.25268	P	PPL	IE		L	07			0		12	Europe/Dublin	2010-08-14
+2961588	Sandymount	Sandymount		53.97139	-6.36194	P	PPL	IE		L	19			0		-9999	Europe/Dublin	2010-08-14
+2961589	Sandymount	Sandymount		53.335	-6.21139	P	PPL	IE		L	07			0		-9999	Europe/Dublin	2010-08-14
+2961590	Sandy Ford	Sandy Ford		52.85194	-6.46139	P	PPL	IE		L	31			0		289	Europe/Dublin	2010-08-14
+2961591	Sandy Cove	Sandy Cove		51.67694	-8.54917	P	PPL	IE		M	04			0		72	Europe/Dublin	2010-08-14
+2961592	Sandeel Bay	Sandeel Bay		52.1575	-6.87	H	BAY	IE		L	30			0		-9999	Europe/Dublin	2010-08-14
+2961593	Samphire Island	Samphire Island		52.27167	-9.865	T	ISL	IE		M	11			0		-9999	Europe/Dublin	2010-08-14
+2961594	Saltmills	Saltmills		52.22194	-6.82667	P	PPL	IE		L	30			0		-9999	Europe/Dublin	2010-08-14
+2961595	Salt Hill	Salt Hill		53.26139	-9.06944	P	PPL	IE		C	10			0		1	Europe/Dublin	2010-08-14
+2961596	Salt Hill	Salt Hill		54.63111	-8.21111	S	EST	IE		U	06			0		-9999	Europe/Dublin	2010-08-14
+2961597	Salterstown	Salterstown		53.87167	-6.30694	P	PPL	IE		L	19			0		1	Europe/Dublin	2010-08-14
+2961598	Saltee Islands	Saltee Islands	Saltee Islands	52.125	-6.59722	T	ISLS	IE		L	30			0		-9999	Europe/Dublin	2010-08-14
+2961599	Little Saltee	Little Saltee	Little Saltee,North Saltee,Saltee Island Little	52.13333	-6.61667	T	ISL	IE	IE	L	30			0		-9999	Europe/Dublin	2010-08-14
+2961600	Great Saltee	Great Saltee	Great Saltee,Saltee Island Great,South Saltee	52.11667	-6.61667	T	ISL	IE	IE	L	30			0		-9999	Europe/Dublin	2010-08-14
+2961601	Lough Salt	Lough Salt		55.08278	-7.80611	H	LK	IE		U	06			0		340	Europe/Dublin	2010-08-14
+2961602	Salrock	Salrock		53.6	-9.85	P	PPL	IE		C	10			0		144	Europe/Dublin	1993-12-27
+2961603	Sallymount House	Sallymount House	Sallymount,Sallymount House	53.12222	-6.70972	S	EST	IE	IE	L	12			0		151	Europe/Dublin	2010-08-14
+2961604	Sally Gap	Sally Gap	Sally Gap,Sully Gap	53.13611	-6.31306	T	GAP	IE	IE	L	31			0		460	Europe/Dublin	2010-08-14
+2961605	Sallybrook Station	Sallybrook Station	Sallybrook,Sallybrook Station	54.95	-7.56667	S	RSTN	IE		U	06			0		86	Europe/Dublin	2010-08-10
+2961606	Sallybrook	Sallybrook	Sallybrook	51.935	-8.38639	P	PPL	IE		M	04			0		76	Europe/Dublin	2010-08-14
+2961607	Sallins	Sallins	Na Sollain,Na Soll�in,Sallins	53.24889	-6.66611	P	PPL	IE		L	12			3164		97	Europe/Dublin	2010-08-14
+2961608	Salisbury Lodge	Salisbury Lodge		53.65	-8	S	EST	IE		L	18			0		79	Europe/Dublin	1993-12-27
+2961609	Salia	Salia		53.95111	-9.94806	P	PPL	IE		C	20			0		26	Europe/Dublin	2010-08-14
+2961610	Saleen Harbour	Saleen Harbour	Saleen Bay,Saleen Harbour	54.18361	-10.03	H	HBR	IE	IE	C	20			0		-9999	Europe/Dublin	2010-08-14
+2961611	Saleen	Saleen		52.55861	-9.46444	P	PPL	IE		M	11			0		11	Europe/Dublin	2010-08-14
+2961612	Saleen	Saleen		51.86333	-8.16556	P	PPL	IE		M	04			0		74	Europe/Dublin	2010-08-14
+2961613	Saivnose River	Saivnose River		51.61278	-9.27	H	STM	IE		M	04			0		56	Europe/Dublin	2010-08-14
+2961614	Saint Thomas Island	Saint Thomas Island		52.69	-8.62	T	ISL	IE		M	03			0		2	Europe/Dublin	2010-08-14
+2961615	Saint Stephens Green Park	Saint Stephens Green Park		53.33784	-6.25813	L	PRK	IE		L	07			0		5	Europe/Dublin	2010-08-14
+2961616	Saints Island	Saints Island		54.61528	-7.88639	T	ISL	IE		U	06			0		152	Europe/Dublin	2010-08-14
+2961617	Patrickswell	Patrickswell	Patrickswell,Saint Patrickswell,Tobar Phadraig,Tobar Ph�draig	52.59722	-8.70889	P	PPL	IE		M	16			0		55	Europe/Dublin	2010-08-14
+2961618	Saint Patricks Island	Saint Patricks Island		53.58833	-6.07389	T	ISL	IE		L	07			0		-9999	Europe/Dublin	2010-08-14
+2961619	Saint Mullins	Saint Mullins		52.49278	-6.92306	P	PPL	IE		L	01			0		45	Europe/Dublin	2010-08-14
+2961620	Saint Margarets House	Saint Margarets House		52.20083	-6.34472	S	BLDG	IE		L	30			0		-9999	Europe/Dublin	2010-08-14
+2961621	Saint Macdaras Island	Saint Macdaras Island	Macdara,Saint Macdaras Island	53.305	-9.91889	T	ISL	IE	IE	C	10			0		-9999	Europe/Dublin	2010-08-14
+2961622	Saint Johnstown	Saint Johnstown	Baile Suingean,Saint Johnston,Saint Johnstown	54.93333	-7.45	P	PPL	IE		U	06			0		1	Europe/Dublin	2010-08-14
+2961623	Saint Johns Port	Saint Johns Port		54.33333	-8.6	P	PPL	IE		C	25			0		-9999	Europe/Dublin	1993-12-27
+2961624	Saint John\u2019s Point	Saint John's Point		54.56667	-8.46667	T	PT	IE		U	06			0		-9999	Europe/Dublin	2010-08-14
+2961625	Saint Johns Lough	Saint Johns Lough		54.04028	-7.86167	H	LK	IE		C	14			0		70	Europe/Dublin	2010-08-14
+2961626	Saint Johns House	Saint Johns House		52.48833	-6.56944	S	EST	IE		L	30			0		51	Europe/Dublin	2010-08-14
+2961627	Saint Finans Bay	Saint Finans Bay	Saint Finans Bay,Saint Finian Bay,Saint Finnans Bay	51.81667	-10.36667	H	BAY	IE	IE	M	11			0		-9999	Europe/Dublin	2010-08-14
+2961628	Saint Edmonds	Saint Edmonds		52.38333	-6.48333	S	EST	IE		L	30			0		1	Europe/Dublin	2010-08-14
+2961629	Saint Doolaghs	Saint Doolaghs		53.41639	-6.18056	P	PPL	IE		L	07			0		3	Europe/Dublin	2010-08-14
+2961630	Saint Clerans	Saint Clerans	Saint Clerans,Saint Clernans	53.22667	-8.65778	S	EST	IE	IE	C	10			0		51	Europe/Dublin	2010-08-14
+2961631	Saint Catherines	Saint Catherines	Saint Catharine's,Saint Catharine\u2019s,Saint Catherines	52.86667	-8.6	P	PPL	IE		M	03			0		76	Europe/Dublin	2010-08-10
+2961632	Saint Brendans House	Saint Brendans House		53.53333	-8.4	S	EST	IE		C	10			0		69	Europe/Dublin	1993-12-27
+2961633	Saint Brendans	Saint Brendans	Saint Bredan's,Saint Bredan\u2019s,Saint Brendans	53.23417	-8.06083	S	EST	IE	IE	C	10			0		61	Europe/Dublin	2010-08-14
+2961634	Saint Anns Wells	Saint Anns Wells		53.23333	-6.35	P	PPL	IE		00				0		300	Europe/Dublin	2007-05-30
+2961635	Saint Anns Hill	Saint Anns Hill	Saint Ann's,Saint Anne's Hill,Saint Anne\u2019s Hill,Saint Anns Hill,Saint Ann\u2019s	51.93333	-8.6	P	PPL	IE	IE	M	04			0		68	Europe/Dublin	2010-08-14
+2961636	Saggart	Saggart	Saggart,Teach Sagard	53.28028	-6.44444	P	PPL	IE	IE	L	07			0		150	Europe/Dublin	2010-11-04
+2961637	Safe Harbour	Safe Harbour		53.53333	-8	H	COVE	IE		C	24			0		62	Europe/Dublin	1993-12-27
+2961638	Saddle Hill	Saddle Hill		54.34861	-8.12917	T	MT	IE		C	14			0	379	310	Europe/Dublin	2010-08-14
+2961639	Saddle Head	Saddle Head	Gubroenacoragh,Saddle Head	54.01222	-10.18889	T	CAPE	IE	IE	C	20			0		45	Europe/Dublin	2010-08-14
+2961640	Ryves Castle	Ryves Castle		52.42306	-8.36611	S	EST	IE		M	16			0		121	Europe/Dublin	2010-08-14
+2961641	Ryndville	Ryndville		53.43667	-6.82306	S	EST	IE		L	21			0		86	Europe/Dublin	2010-08-14
+2961642	Rylane Cross	Rylane Cross	Rylane,Rylane Cross	51.98333	-8.83333	P	PPL	IE	IE	M	04			0		191	Europe/Dublin	2010-08-14
+2961643	Rye Water	Rye Water	An Ri,Rye Water	53.365	-6.49111	H	STM	IE	IE	00				0		47	Europe/Dublin	2010-08-10
+2961644	Ryehill	Ryehill		53.39889	-8.69222	S	EST	IE		C	10			0		76	Europe/Dublin	2010-08-14
+2961645	Ryefield	Ryefield		53.78806	-7.05861	P	PPL	IE		U	02			0		126	Europe/Dublin	2010-08-14
+2961646	Rutland Island	Rutland Island		54.97667	-8.45722	T	ISL	IE		U	06			0		1	Europe/Dublin	2010-08-14
+2961647	Russborough House	Russborough House	Russborough,Russborough House	53.14111	-6.56806	S	HSEC	IE	IE	L	31			0		181	Europe/Dublin	2010-08-14
+2961648	Rushin House	Rushin House		53.01778	-7.49528	S	EST	IE		L	15			0		129	Europe/Dublin	2010-08-14
+2961649	Slieve Rushen	Slieve Rushen	Slieve Rushen	54.15	-7.63333	T	MT	IE		U	02			0		309	Europe/Dublin	1998-02-13
+2961650	Rushbrooke	Rushbrooke	Rushbrooke	51.85	-8.31667	P	PPL	IE		M	04			0		-9999	Europe/Dublin	2010-08-14
+2961651	An Ros	An Ros	Rush	53.5223	-6.09308	P	PPL	IE		L	07			7294		1	Europe/Dublin	2010-08-14
+2961652	Runnamoat House	Runnamoat House	Runnamoat,Runnamoat House	53.68333	-8.31667	S	EST	IE		C	24			0		77	Europe/Dublin	2010-08-10
+2961653	Rue Point	Rue Point	Bankmore Point,Rue Point	54.43639	-8.64583	T	PT	IE	IE	C	25			0		-9999	Europe/Dublin	2010-08-14
+2961654	Ruan	Ruan		52.93056	-8.98972	P	PPL	IE		M	03			0		38	Europe/Dublin	2010-08-14
+2961655	Royaloak	Royaloak		52.7	-6.98667	P	PPL	IE		L	01			0		40	Europe/Dublin	2010-08-14
+2961656	Royal Canal	Royal Canal	An Chanail Rioga,Royal Canal	53.35	-6.23333	H	CNL	IE	IE	00				0		1	Europe/Dublin	2010-08-10
+2961657	Roy	Roy		54.08028	-9.94528	P	PPLL	IE		C	20			0		1	Europe/Dublin	2010-08-14
+2961658	Roxborough House	Roxborough House	Roxborough,Roxborough House	53.65	-8.18333	S	EST	IE		C	24			0		75	Europe/Dublin	2010-08-10
+2961659	Roxborough House	Roxborough House	Roxborough,Roxborough House	52.61306	-8.615	S	EST	IE	IE	M	16			0		56	Europe/Dublin	2010-08-14
+2961660	Roxborough	Roxborough		53.16694	-8.69528	S	EST	IE		C	10			0		76	Europe/Dublin	2010-08-14
+2961661	Rowls Aldworth	Rowls Aldworth		52.3	-9.01667	S	BLDG

<TRUNCATED>

[15/26] lucenenet git commit: Added missing Spatial classes, ported Spatial tests, and fixed several bugs

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Lucene.Net.Tests.Spatial.csproj
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Lucene.Net.Tests.Spatial.csproj b/src/Lucene.Net.Tests.Spatial/Lucene.Net.Tests.Spatial.csproj
new file mode 100644
index 0000000..d74a255
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Lucene.Net.Tests.Spatial.csproj
@@ -0,0 +1,130 @@
+\ufeff<?xml version="1.0" encoding="utf-8"?>
+<Project ToolsVersion="14.0" DefaultTargets="Build" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
+  <Import Project="$(MSBuildExtensionsPath)\$(MSBuildToolsVersion)\Microsoft.Common.props" Condition="Exists('$(MSBuildExtensionsPath)\$(MSBuildToolsVersion)\Microsoft.Common.props')" />
+  <PropertyGroup>
+    <Configuration Condition=" '$(Configuration)' == '' ">Debug</Configuration>
+    <Platform Condition=" '$(Platform)' == '' ">AnyCPU</Platform>
+    <ProjectGuid>{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}</ProjectGuid>
+    <OutputType>Library</OutputType>
+    <AppDesignerFolder>Properties</AppDesignerFolder>
+    <RootNamespace>Lucene.Net.Tests.Spatial</RootNamespace>
+    <AssemblyName>Lucene.Net.Tests.Spatial</AssemblyName>
+    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <FileAlignment>512</FileAlignment>
+  </PropertyGroup>
+  <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Debug|AnyCPU' ">
+    <DebugSymbols>true</DebugSymbols>
+    <DebugType>full</DebugType>
+    <Optimize>false</Optimize>
+    <OutputPath>bin\Debug\</OutputPath>
+    <DefineConstants>DEBUG;TRACE</DefineConstants>
+    <ErrorReport>prompt</ErrorReport>
+    <WarningLevel>4</WarningLevel>
+  </PropertyGroup>
+  <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Release|AnyCPU' ">
+    <DebugType>pdbonly</DebugType>
+    <Optimize>true</Optimize>
+    <OutputPath>bin\Release\</OutputPath>
+    <DefineConstants>TRACE</DefineConstants>
+    <ErrorReport>prompt</ErrorReport>
+    <WarningLevel>4</WarningLevel>
+  </PropertyGroup>
+  <ItemGroup>
+    <Reference Include="nunit.framework, Version=2.6.3.13283, Culture=neutral, PublicKeyToken=96d09a1eb7f44a77, processorArchitecture=MSIL">
+      <HintPath>..\..\packages\NUnit.2.6.3\lib\nunit.framework.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="Spatial4n.Core.NTS">
+      <HintPath>..\..\packages\Spatial4n.Core.NTS.0.4.1\lib\net40\Spatial4n.Core.NTS.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="GeoAPI, Version=1.7.4.0, Culture=neutral, PublicKeyToken=a1a0da7def465678, processorArchitecture=MSIL">
+      <HintPath>..\..\packages\GeoAPI.1.7.4\lib\net45\GeoAPI.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="NetTopologySuite, Version=1.14.0.0, Culture=neutral, PublicKeyToken=f580a05016ebada1, processorArchitecture=MSIL">
+      <HintPath>..\..\packages\NetTopologySuite.1.14\lib\net45\NetTopologySuite.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="PowerCollections, Version=1.0.0.0, Culture=neutral, PublicKeyToken=2573bf8a1bdddcd5, processorArchitecture=MSIL">
+      <HintPath>..\..\packages\NetTopologySuite.1.14\lib\net45\PowerCollections.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="System" />
+    <Reference Include="System.Core" />
+    <Reference Include="System.Xml.Linq" />
+    <Reference Include="System.Data.DataSetExtensions" />
+    <Reference Include="Microsoft.CSharp" />
+    <Reference Include="System.Data" />
+    <Reference Include="System.Net.Http" />
+    <Reference Include="System.Xml" />
+  </ItemGroup>
+  <ItemGroup>
+    <Compile Include="DistanceStrategyTest.cs" />
+    <Compile Include="PortedSolr3Test.cs" />
+    <Compile Include="Prefix\NtsPolygonTest.cs" />
+    <Compile Include="Prefix\SpatialOpRecursivePrefixTreeTest.cs" />
+    <Compile Include="Prefix\TestRecursivePrefixTreeStrategy.cs" />
+    <Compile Include="Prefix\TestTermQueryPrefixGridStrategy.cs" />
+    <Compile Include="Prefix\Tree\SpatialPrefixTreeTest.cs" />
+    <Compile Include="Properties\AssemblyInfo.cs" />
+    <Compile Include="QueryEqualsHashCodeTest.cs" />
+    <Compile Include="Query\SpatialArgsParserTest.cs" />
+    <Compile Include="Serialized\SerializedStrategyTest.cs" />
+    <Compile Include="SpatialArgsTest.cs" />
+    <Compile Include="SpatialExample.cs" />
+    <Compile Include="SpatialMatchConcern.cs" />
+    <Compile Include="SpatialTestCase.cs" />
+    <Compile Include="SpatialTestData.cs" />
+    <Compile Include="SpatialTestQuery.cs" />
+    <Compile Include="StrategyTestCase.cs" />
+    <Compile Include="TestTestFramework.cs" />
+    <Compile Include="Vector\TestPointVectorStrategy.cs" />
+  </ItemGroup>
+  <ItemGroup>
+    <ProjectReference Include="..\Lucene.Net.Core\Lucene.Net.csproj">
+      <Project>{5d4ad9be-1ffb-41ab-9943-25737971bf57}</Project>
+      <Name>Lucene.Net</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\Lucene.Net.Queries\Lucene.Net.Queries.csproj">
+      <Project>{69D7956C-C2CC-4708-B399-A188FEC384C4}</Project>
+      <Name>Lucene.Net.Queries</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\Lucene.Net.Spatial\Lucene.Net.Spatial.csproj">
+      <Project>{35c347f4-24b2-4be5-8117-a0e3001551ce}</Project>
+      <Name>Lucene.Net.Spatial</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\Lucene.Net.TestFramework\Lucene.Net.TestFramework.csproj">
+      <Project>{B2C0D749-CE34-4F62-A15E-00CB2FF5DDB3}</Project>
+      <Name>Lucene.Net.TestFramework</Name>
+    </ProjectReference>
+  </ItemGroup>
+  <ItemGroup>
+    <None Include="packages.config" />
+  </ItemGroup>
+  <ItemGroup>
+    <Service Include="{82A7F48D-3B50-4B1E-B82E-3ADA8210C358}" />
+  </ItemGroup>
+  <ItemGroup>
+    <EmbeddedResource Include="Test-Files\cities-Intersects-BBox.txt" />
+    <EmbeddedResource Include="Test-Files\Data\countries-bbox.txt" />
+    <EmbeddedResource Include="Test-Files\Data\countries-poly.txt" />
+    <EmbeddedResource Include="Test-Files\Data\geonames-IE.txt" />
+    <EmbeddedResource Include="Test-Files\Data\LUCENE-4464.txt" />
+    <EmbeddedResource Include="Test-Files\Data\simple-bbox.txt" />
+    <EmbeddedResource Include="Test-Files\Data\states-bbox.txt" />
+    <EmbeddedResource Include="Test-Files\Data\states-poly.txt" />
+    <EmbeddedResource Include="Test-Files\Data\world-cities-points.txt" />
+    <EmbeddedResource Include="Test-Files\simple-Queries-BBox.txt" />
+    <EmbeddedResource Include="Test-Files\states-Intersects-BBox.txt" />
+    <EmbeddedResource Include="Test-Files\states-IsWithin-BBox.txt" />
+  </ItemGroup>
+  <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
+  <!-- To modify your build process, add your task inside one of the targets below and uncomment it. 
+       Other similar extension points exist, see Microsoft.Common.targets.
+  <Target Name="BeforeBuild">
+  </Target>
+  <Target Name="AfterBuild">
+  </Target>
+  -->
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/PortedSolr3Test.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/PortedSolr3Test.cs b/src/Lucene.Net.Tests.Spatial/PortedSolr3Test.cs
new file mode 100644
index 0000000..e120b0b
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/PortedSolr3Test.cs
@@ -0,0 +1,201 @@
+\ufeffusing Lucene.Net.Randomized.Generators;
+using Lucene.Net.Search;
+using Lucene.Net.Spatial.Prefix;
+using Lucene.Net.Spatial.Prefix.Tree;
+using Lucene.Net.Spatial.Queries;
+using Lucene.Net.Spatial.Vector;
+using NUnit.Framework;
+using Spatial4n.Core.Context;
+using Spatial4n.Core.Distance;
+using Spatial4n.Core.Shapes;
+using System;
+using System.Collections.Generic;
+using System.Globalization;
+
+namespace Lucene.Net.Spatial
+{
+    /*
+     * 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.
+     */
+
+    /// <summary>
+    /// Based off of Solr 3's SpatialFilterTest.
+    /// </summary>
+    public class PortedSolr3Test : StrategyTestCase
+    {
+        //@ParametersFactory
+        public static IList<Object[]> Parameters()
+        {
+            List<Object[]> ctorArgs = new List<object[]>();
+
+            SpatialContext ctx = SpatialContext.GEO;
+            SpatialPrefixTree grid;
+            SpatialStrategy strategy;
+
+            grid = new GeohashPrefixTree(ctx, 12);
+            strategy = new RecursivePrefixTreeStrategy(grid, "recursive_geohash");
+            ctorArgs.Add(new Object[] { new Param(strategy) });
+
+            grid = new QuadPrefixTree(ctx, 25);
+            strategy = new RecursivePrefixTreeStrategy(grid, "recursive_quad");
+            ctorArgs.Add(new Object[] { new Param(strategy) });
+
+            grid = new GeohashPrefixTree(ctx, 12);
+            strategy = new TermQueryPrefixTreeStrategy(grid, "termquery_geohash");
+            ctorArgs.Add(new Object[] { new Param(strategy) });
+
+            strategy = new PointVectorStrategy(ctx, "pointvector");
+            ctorArgs.Add(new Object[] { new Param(strategy) });
+
+            return ctorArgs;
+        }
+
+        // this is a hack for clover! (otherwise strategy.toString() used as file name)
+        public class Param
+        {
+            internal SpatialStrategy strategy;
+
+            internal Param(SpatialStrategy strategy) { this.strategy = strategy; }
+
+
+            public override String ToString() { return strategy.FieldName; }
+        }
+
+        //  private String fieldName;
+
+        //public PortedSolr3Test(Param param)
+        //{
+        //    SpatialStrategy strategy = param.strategy;
+        //    this.ctx = strategy.SpatialContext;
+        //    this.strategy = strategy;
+        //}
+
+        public override void SetUp()
+        {
+            base.SetUp();
+            SpatialStrategy strategy = ((Param)(RandomInts.RandomFrom(Random(), Parameters()))[0]).strategy;
+            this.ctx = strategy.SpatialContext;
+            this.strategy = strategy;
+        }
+
+        private void setupDocs()
+        {
+            base.DeleteAll();
+            adoc("1", ctx.MakePoint(-79.9289094, 32.7693246));
+            adoc("2", ctx.MakePoint(-80.9289094, 33.7693246));
+            adoc("3", ctx.MakePoint(50.9289094, -32.7693246));
+            adoc("4", ctx.MakePoint(60.9289094, -50.7693246));
+            adoc("5", ctx.MakePoint(0, 0));
+            adoc("6", ctx.MakePoint(0.1, 0.1));
+            adoc("7", ctx.MakePoint(-0.1, -0.1));
+            adoc("8", ctx.MakePoint(179.9, 0));
+            adoc("9", ctx.MakePoint(-179.9, 0));
+            adoc("10", ctx.MakePoint(50, 89.9));
+            adoc("11", ctx.MakePoint(-130, 89.9));
+            adoc("12", ctx.MakePoint(50, -89.9));
+            adoc("13", ctx.MakePoint(-130, -89.9));
+            Commit();
+        }
+
+
+        [Test]
+        public virtual void TestIntersections()
+        {
+            setupDocs();
+            //Try some edge cases
+            //NOTE: 2nd arg is distance in kilometers
+            CheckHitsCircle(ctx.MakePoint(1, 1), 175, 3, 5, 6, 7);
+            CheckHitsCircle(ctx.MakePoint(179.8, 0), 200, 2, 8, 9);
+            CheckHitsCircle(ctx.MakePoint(50, 89.8), 200, 2, 10, 11);//this goes over the north pole
+            CheckHitsCircle(ctx.MakePoint(50, -89.8), 200, 2, 12, 13);//this goes over the south pole
+                                                                      //try some normal cases
+            CheckHitsCircle(ctx.MakePoint(-80.0, 33.0), 300, 2);
+            //large distance
+            CheckHitsCircle(ctx.MakePoint(1, 1), 5000, 3, 5, 6, 7);
+            //Because we are generating a box based on the west/east longitudes and the south/north latitudes, which then
+            //translates to a range query, which is slightly more inclusive.  Thus, even though 0.0 is 15.725 kms away,
+            //it will be included, b/c of the box calculation.
+            CheckHitsBBox(ctx.MakePoint(0.1, 0.1), 15, 2, 5, 6);
+            //try some more
+            DeleteAll();
+            adoc("14", ctx.MakePoint(5, 0));
+            adoc("15", ctx.MakePoint(15, 0));
+            //3000KM from 0,0, see http://www.movable-type.co.uk/scripts/latlong.html
+            adoc("16", ctx.MakePoint(19.79750, 18.71111));
+            adoc("17", ctx.MakePoint(-95.436643, 44.043900));
+            Commit();
+
+            CheckHitsCircle(ctx.MakePoint(0, 0), 1000, 1, 14);
+            CheckHitsCircle(ctx.MakePoint(0, 0), 2000, 2, 14, 15);
+            CheckHitsBBox(ctx.MakePoint(0, 0), 3000, 3, 14, 15, 16);
+            CheckHitsCircle(ctx.MakePoint(0, 0), 3001, 3, 14, 15, 16);
+            CheckHitsCircle(ctx.MakePoint(0, 0), 3000.1, 3, 14, 15, 16);
+
+            //really fine grained distance and reflects some of the vagaries of how we are calculating the box
+            CheckHitsCircle(ctx.MakePoint(-96.789603, 43.517030), 109, 0);
+
+            // falls outside of the real distance, but inside the bounding box
+            CheckHitsCircle(ctx.MakePoint(-96.789603, 43.517030), 110, 0);
+            CheckHitsBBox(ctx.MakePoint(-96.789603, 43.517030), 110, 1, 17);
+        }
+
+        //---- these are similar to Solr test methods
+
+        private void CheckHitsCircle(IPoint pt, double distKM, int assertNumFound, params int[] assertIds)
+        {
+            _CheckHits(false, pt, distKM, assertNumFound, assertIds);
+        }
+        private void CheckHitsBBox(IPoint pt, double distKM, int assertNumFound, params int[] assertIds)
+        {
+            _CheckHits(true, pt, distKM, assertNumFound, assertIds);
+        }
+
+        private void _CheckHits(bool bbox, IPoint pt, double distKM, int assertNumFound, params int[] assertIds)
+        {
+            SpatialOperation op = SpatialOperation.Intersects;
+            double distDEG = DistanceUtils.Dist2Degrees(distKM, DistanceUtils.EARTH_MEAN_RADIUS_KM);
+            IShape shape = ctx.MakeCircle(pt, distDEG);
+            if (bbox)
+                shape = shape.BoundingBox;
+
+            SpatialArgs args = new SpatialArgs(op, shape);
+            //args.setDistPrecision(0.025);
+            Query query;
+            if (Random().nextBoolean())
+            {
+                query = strategy.MakeQuery(args);
+            }
+            else
+            {
+                query = new FilteredQuery(new MatchAllDocsQuery(), strategy.MakeFilter(args));
+            }
+            SearchResults results = executeQuery(query, 100);
+            assertEquals("" + shape, assertNumFound, results.numFound);
+            if (assertIds != null)
+            {
+                ISet<int?> resultIds = new HashSet<int?>();
+                foreach (SearchResult result in results.results)
+                {
+                    resultIds.add(int.Parse(result.document.Get("id"), CultureInfo.InvariantCulture));
+                }
+                foreach (int assertId in assertIds)
+                {
+                    assertTrue("has " + assertId, resultIds.contains(assertId));
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Prefix/NtsPolygonTest.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Prefix/NtsPolygonTest.cs b/src/Lucene.Net.Tests.Spatial/Prefix/NtsPolygonTest.cs
new file mode 100644
index 0000000..7f4338d
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Prefix/NtsPolygonTest.cs
@@ -0,0 +1,118 @@
+\ufeffusing Lucene.Net.Documents;
+using Lucene.Net.Search;
+using Lucene.Net.Spatial.Prefix.Tree;
+using Lucene.Net.Spatial.Queries;
+using Lucene.Net.Support;
+using NUnit.Framework;
+using Spatial4n.Core.Context;
+using Spatial4n.Core.Context.Nts;
+using Spatial4n.Core.Shapes;
+using System;
+using System.Collections.Generic;
+
+namespace Lucene.Net.Spatial.Prefix
+{
+    /*
+     * 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.
+     */
+
+    public class NtsPolygonTest : StrategyTestCase
+    {
+        private static readonly double LUCENE_4464_distErrPct = SpatialArgs.DEFAULT_DISTERRPCT;//DEFAULT 2.5%
+
+        public NtsPolygonTest()
+        {
+            try
+            {
+                IDictionary<string, string> args = new Dictionary<string, string>();
+                args.Put("spatialContextFactory",
+                    typeof(NtsSpatialContextFactory).AssemblyQualifiedName);
+                ctx = SpatialContextFactory.MakeSpatialContext(args /*, getClass().getClassLoader()*/);
+            }
+            catch (TypeLoadException e) //LUCENENET TODO: Does this match NoClassDefFoundError ??
+            {
+                AssumeTrue("This test requires JTS jar: " + e, false);
+            }
+
+            GeohashPrefixTree grid = new GeohashPrefixTree(ctx, 11);//< 1 meter == 11 maxLevels
+            this.strategy = new RecursivePrefixTreeStrategy(grid, GetType().Name);
+            ((RecursivePrefixTreeStrategy)this.strategy).DistErrPct = (LUCENE_4464_distErrPct);//1% radius (small!)
+        }
+
+        [Test]
+        /** LUCENE-4464 */
+        public virtual void TestCloseButNoMatch()
+        {
+            getAddAndVerifyIndexedDocuments("LUCENE-4464.txt");
+            SpatialArgs args = q(
+                "POLYGON((-93.18100824442227 45.25676372469945," +
+                    "-93.23182001200654 45.21421290799412," +
+                    "-93.16315546122038 45.23742639412364," +
+                    "-93.18100824442227 45.25676372469945))",
+                LUCENE_4464_distErrPct);
+            SearchResults got = executeQuery(strategy.MakeQuery(args), 100);
+            assertEquals(1, got.numFound);
+            assertEquals("poly2", got.results[0].document.Get("id"));
+            //did not find poly 1 !
+        }
+
+        private SpatialArgs q(String shapeStr, double distErrPct)
+        {
+            IShape shape = ctx.ReadShapeFromWkt(shapeStr);
+            SpatialArgs args = new SpatialArgs(SpatialOperation.Intersects, shape);
+            args.DistErrPct = (distErrPct);
+            return args;
+        }
+
+        /**
+         * A PrefixTree pruning optimization gone bad.
+         * See <a href="https://issues.apache.org/jira/browse/LUCENE-4770>LUCENE-4770</a>.
+         */
+        [Test]
+        public virtual void TestBadPrefixTreePrune()
+        {
+
+            IShape area = ctx.ReadShapeFromWkt("POLYGON((-122.83 48.57, -122.77 48.56, -122.79 48.53, -122.83 48.57))");
+
+            SpatialPrefixTree trie = new QuadPrefixTree(ctx, 12);
+            TermQueryPrefixTreeStrategy strategy = new TermQueryPrefixTreeStrategy(trie, "geo");
+            Document doc = new Document();
+            doc.Add(new TextField("id", "1", Field.Store.YES)); // LUCENENET TODO: Change API to Store.Yes
+
+            Field[] fields = strategy.CreateIndexableFields(area, 0.025);
+            foreach (Field field in fields)
+            {
+                doc.Add(field);
+            }
+            AddDocument(doc);
+
+            IPoint upperleft = ctx.MakePoint(-122.88, 48.54);
+            IPoint lowerright = ctx.MakePoint(-122.82, 48.62);
+
+            Query query = strategy.MakeQuery(new SpatialArgs(SpatialOperation.Intersects, ctx.MakeRectangle(upperleft, lowerright)));
+            Commit();
+
+            TopDocs search = indexSearcher.Search(query, 10);
+            ScoreDoc[] scoreDocs = search.ScoreDocs;
+            foreach (ScoreDoc scoreDoc in scoreDocs)
+            {
+                Console.WriteLine(indexSearcher.Doc(scoreDoc.Doc));
+            }
+
+            assertEquals(1, search.TotalHits);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Prefix/SpatialOpRecursivePrefixTreeTest.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Prefix/SpatialOpRecursivePrefixTreeTest.cs b/src/Lucene.Net.Tests.Spatial/Prefix/SpatialOpRecursivePrefixTreeTest.cs
new file mode 100644
index 0000000..d7d3fb7
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Prefix/SpatialOpRecursivePrefixTreeTest.cs
@@ -0,0 +1,485 @@
+\ufeffusing Lucene.Net.Documents;
+using Lucene.Net.Search;
+using Lucene.Net.Spatial.Prefix.Tree;
+using Lucene.Net.Spatial.Queries;
+using Lucene.Net.Support;
+using NUnit.Framework;
+using Spatial4n.Core.Context;
+using Spatial4n.Core.Shapes;
+using Spatial4n.Core.Shapes.Impl;
+using System;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Lucene.Net.Spatial.Prefix
+{
+    /*
+     * 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.
+     */
+
+    public class SpatialOpRecursivePrefixTreeTest : StrategyTestCase
+    {
+        const int ITERATIONS = 1;//Test Iterations
+
+        private SpatialPrefixTree grid;
+
+
+        public override void SetUp()
+        {
+            base.SetUp();
+            DeleteAll();
+        }
+
+        public virtual void SetupGrid(int maxLevels)
+        {
+            if (Random().nextBoolean())
+                SetupQuadGrid(maxLevels);
+            else
+                SetupGeohashGrid(maxLevels);
+            //((PrefixTreeStrategy) strategy).setDistErrPct(0);//fully precise to grid
+
+            Console.WriteLine("Strategy: " + strategy.toString());
+        }
+
+        private void SetupQuadGrid(int maxLevels)
+        {
+            //non-geospatial makes this test a little easier (in gridSnap), and using boundary values 2^X raises
+            // the prospect of edge conditions we want to test, plus makes for simpler numbers (no decimals).
+            SpatialContextFactory factory = new SpatialContextFactory();
+            factory.geo = false;
+            factory.worldBounds = new Rectangle(0, 256, -128, 128, null);
+            this.ctx = factory.NewSpatialContext();
+            //A fairly shallow grid, and default 2.5% distErrPct
+            if (maxLevels == -1)
+                maxLevels = randomIntBetween(1, 8);//max 64k cells (4^8), also 256*256
+            this.grid = new QuadPrefixTree(ctx, maxLevels);
+            this.strategy = new RecursivePrefixTreeStrategy(grid, GetType().Name);
+        }
+
+        public virtual void SetupGeohashGrid(int maxLevels)
+        {
+            this.ctx = SpatialContext.GEO;
+            //A fairly shallow grid, and default 2.5% distErrPct
+            if (maxLevels == -1)
+                maxLevels = randomIntBetween(1, 3);//max 16k cells (32^3)
+            this.grid = new GeohashPrefixTree(ctx, maxLevels);
+            this.strategy = new RecursivePrefixTreeStrategy(grid, GetType().Name);
+        }
+
+        [Test, Repeat(ITERATIONS)]
+        public virtual void TestIntersects()
+        {
+            SetupGrid(-1);
+            doTest(SpatialOperation.Intersects);
+        }
+
+        [Test, Repeat(ITERATIONS)]
+        public virtual void TestWithin()
+        {
+            SetupGrid(-1);
+            doTest(SpatialOperation.IsWithin);
+        }
+
+        [Test, Repeat(ITERATIONS)]
+        public virtual void TestContains()
+        {
+            SetupGrid(-1);
+            doTest(SpatialOperation.Contains);
+        }
+
+        [Test, Repeat(ITERATIONS)]
+        public virtual void TestDisjoint()
+        {
+            SetupGrid(-1);
+            doTest(SpatialOperation.IsDisjointTo);
+        }
+
+        /** See LUCENE-5062, {@link ContainsPrefixTreeFilter#multiOverlappingIndexedShapes}. */
+        [Test, Repeat(ITERATIONS)]
+        public virtual void TestContainsPairOverlap()
+        {
+            SetupQuadGrid(3);
+            adoc("0", new ShapePair(ctx.MakeRectangle(0, 33, -128, 128), ctx.MakeRectangle(33, 128, -128, 128), true, ctx));
+            Commit();
+            Query query = strategy.MakeQuery(new SpatialArgs(SpatialOperation.Contains,
+                ctx.MakeRectangle(0, 128, -16, 128)));
+            SearchResults searchResults = executeQuery(query, 1);
+            assertEquals(1, searchResults.numFound);
+        }
+
+        [Test]
+        public virtual void TestWithinDisjointParts()
+        {
+            SetupQuadGrid(7);
+            //one shape comprised of two parts, quite separated apart
+            adoc("0", new ShapePair(ctx.MakeRectangle(0, 10, -120, -100), ctx.MakeRectangle(220, 240, 110, 125), false, ctx));
+            Commit();
+            //query surrounds only the second part of the indexed shape
+            Query query = strategy.MakeQuery(new SpatialArgs(SpatialOperation.IsWithin,
+                ctx.MakeRectangle(210, 245, 105, 128)));
+            SearchResults searchResults = executeQuery(query, 1);
+            //we shouldn't find it because it's not completely within
+            assertTrue(searchResults.numFound == 0);
+        }
+
+        [Test] /** LUCENE-4916 */
+        public virtual void TestWithinLeafApproxRule()
+        {
+            SetupQuadGrid(2);//4x4 grid
+                             //indexed shape will simplify to entire right half (2 top cells)
+            adoc("0", ctx.MakeRectangle(192, 204, -128, 128));
+            Commit();
+
+            ((RecursivePrefixTreeStrategy)strategy).PrefixGridScanLevel = (Random().nextInt(2 + 1));
+
+            //query does NOT contain it; both indexed cells are leaves to the query, and
+            // when expanded to the full grid cells, the top one's top row is disjoint
+            // from the query and thus not a match.
+            assertTrue(executeQuery(strategy.MakeQuery(
+                new SpatialArgs(SpatialOperation.IsWithin, ctx.MakeRectangle(38, 192, -72, 56))
+            ), 1).numFound == 0);//no-match
+
+            //this time the rect is a little bigger and is considered a match. It's a
+            // an acceptable false-positive because of the grid approximation.
+            assertTrue(executeQuery(strategy.MakeQuery(
+                new SpatialArgs(SpatialOperation.IsWithin, ctx.MakeRectangle(38, 192, -72, 80))
+            ), 1).numFound == 1);//match
+        }
+
+        //Override so we can index parts of a pair separately, resulting in the detailLevel
+        // being independent for each shape vs the whole thing
+        protected override Document newDoc(String id, IShape shape)
+        {
+            Document doc = new Document();
+            doc.Add(new StringField("id", id, Field.Store.YES));
+            if (shape != null)
+            {
+                IList<IShape> shapes;
+                if (shape is ShapePair)
+                {
+                    shapes = new List<IShape>(2);
+                    shapes.Add(((ShapePair)shape).shape1);
+                    shapes.Add(((ShapePair)shape).shape2);
+                }
+                else
+                {
+                    shapes = new List<IShape>(new IShape[] { shape });//Collections.Singleton(shape);
+                }
+                foreach (IShape shapei in shapes)
+                {
+                    foreach (Field f in strategy.CreateIndexableFields(shapei))
+                    {
+                        doc.Add(f);
+                    }
+                }
+                if (storeShape)//just for diagnostics
+                    doc.Add(new StoredField(strategy.FieldName, shape.toString()));
+            }
+            return doc;
+        }
+
+        private void doTest(SpatialOperation operation)
+        {
+            //first show that when there's no data, a query will result in no results
+            {
+                Query query = strategy.MakeQuery(new SpatialArgs(operation, randomRectangle()));
+                SearchResults searchResults = executeQuery(query, 1);
+                assertEquals(0, searchResults.numFound);
+            }
+
+            bool biasContains = (operation == SpatialOperation.Contains);
+
+            //Main index loop:
+            IDictionary<String, IShape> indexedShapes = new LinkedHashMap<String, IShape>();
+            IDictionary<String, IShape> indexedShapesGS = new LinkedHashMap<String, IShape>();//grid snapped
+            int numIndexedShapes = randomIntBetween(1, 6);
+            bool indexedAtLeastOneShapePair = false;
+            for (int i = 0; i < numIndexedShapes; i++)
+            {
+                String id = "" + i;
+                IShape indexedShape;
+                int R = Random().nextInt(12);
+                if (R == 0)
+                {//1 in 12
+                    indexedShape = null;
+                }
+                else if (R == 1)
+                {//1 in 12
+                    indexedShape = randomPoint();//just one point
+                }
+                else if (R <= 4)
+                {//3 in 12
+                 //comprised of more than one shape
+                    indexedShape = randomShapePairRect(biasContains);
+                    indexedAtLeastOneShapePair = true;
+                }
+                else
+                {
+                    indexedShape = randomRectangle();//just one rect
+                }
+
+                indexedShapes.Put(id, indexedShape);
+                indexedShapesGS.Put(id, gridSnap(indexedShape));
+
+                adoc(id, indexedShape);
+
+                if (Random().nextInt(10) == 0)
+                    Commit();//intermediate commit, produces extra segments
+
+            }
+            //delete some documents randomly
+            IEnumerator<String> idIter = indexedShapes.Keys.ToList().GetEnumerator();
+            while (idIter.MoveNext())
+            {
+                String id = idIter.Current;
+                if (Random().nextInt(10) == 0)
+                {
+                    DeleteDoc(id);
+                    //idIter.Remove();
+                    indexedShapes.Remove(id); // LUCENENET TODO: Verify this works.
+                    indexedShapesGS.Remove(id);
+                }
+            }
+
+            Commit();
+
+            //Main query loop:
+            int numQueryShapes = AtLeast(20);
+            for (int i = 0; i < numQueryShapes; i++)
+            {
+                int scanLevel = randomInt(grid.MaxLevels);
+                ((RecursivePrefixTreeStrategy)strategy).PrefixGridScanLevel = (scanLevel);
+
+                IShape queryShape;
+                switch (randomInt(10))
+                {
+                    case 0: queryShape = randomPoint(); break;
+                    // LUCENE-5549
+                    //TODO debug: -Dtests.method=testWithin -Dtests.multiplier=3 -Dtests.seed=5F5294CE2E075A3E:AAD2F0F79288CA64
+                    //        case 1:case 2:case 3:
+                    //          if (!indexedAtLeastOneShapePair) { // avoids ShapePair.relate(ShapePair), which isn't reliable
+                    //            queryShape = randomShapePairRect(!biasContains);//invert biasContains for query side
+                    //            break;
+                    //          }
+                    default: queryShape = randomRectangle(); break;
+                }
+                IShape queryShapeGS = gridSnap(queryShape);
+
+                bool opIsDisjoint = operation == SpatialOperation.IsDisjointTo;
+
+                //Generate truth via brute force:
+                // We ensure true-positive matches (if the predicate on the raw shapes match
+                //  then the search should find those same matches).
+                // approximations, false-positive matches
+                ISet<String> expectedIds = new /* LinkedHashSet<string>*/ HashSet<string>();//true-positives
+                ISet<String> secondaryIds = new /* LinkedHashSet<string>*/ HashSet<string>();//false-positives (unless disjoint)
+                foreach (var entry in indexedShapes)
+                {
+                    String id = entry.Key;
+                    IShape indexedShapeCompare = entry.Value;
+                    if (indexedShapeCompare == null)
+                        continue;
+                    IShape queryShapeCompare = queryShape;
+
+                    if (operation.Evaluate(indexedShapeCompare, queryShapeCompare))
+                    {
+                        expectedIds.add(id);
+                        if (opIsDisjoint)
+                        {
+                            //if no longer intersect after buffering them, for disjoint, remember this
+                            indexedShapeCompare = indexedShapesGS[id];
+                            queryShapeCompare = queryShapeGS;
+                            if (!operation.Evaluate(indexedShapeCompare, queryShapeCompare))
+                                secondaryIds.add(id);
+                        }
+                    }
+                    else if (!opIsDisjoint)
+                    {
+                        //buffer either the indexed or query shape (via gridSnap) and try again
+                        if (operation == SpatialOperation.Intersects)
+                        {
+                            indexedShapeCompare = indexedShapesGS[id];
+                            queryShapeCompare = queryShapeGS;
+                            //TODO Unfortunately, grid-snapping both can result in intersections that otherwise
+                            // wouldn't happen when the grids are adjacent. Not a big deal but our test is just a
+                            // bit more lenient.
+                        }
+                        else if (operation == SpatialOperation.Contains)
+                        {
+                            indexedShapeCompare = indexedShapesGS[id];
+                        }
+                        else if (operation == SpatialOperation.IsWithin)
+                        {
+                            queryShapeCompare = queryShapeGS;
+                        }
+                        if (operation.Evaluate(indexedShapeCompare, queryShapeCompare))
+                            secondaryIds.add(id);
+                    }
+                }
+
+                //Search and verify results
+                SpatialArgs args = new SpatialArgs(operation, queryShape);
+                if (queryShape is ShapePair)
+                    args.DistErrPct = (0.0);//a hack; we want to be more detailed than gridSnap(queryShape)
+                Query query = strategy.MakeQuery(args);
+                SearchResults got = executeQuery(query, 100);
+                ISet<String> remainingExpectedIds = new /* LinkedHashSet<string>*/ HashSet<string>(expectedIds);
+                foreach (SearchResult result in got.results)
+                {
+                    String id = result.GetId();
+                    bool removed = remainingExpectedIds.remove(id);
+                    if (!removed && (!opIsDisjoint && !secondaryIds.contains(id)))
+                    {
+                        fail("Shouldn't match", id, indexedShapes, indexedShapesGS, queryShape);
+                    }
+                }
+                if (opIsDisjoint)
+                    remainingExpectedIds.removeAll(secondaryIds);
+                if (remainingExpectedIds.Any())
+                {
+                    var iter = remainingExpectedIds.GetEnumerator();
+                    iter.MoveNext();
+                    String id = iter.Current;
+                    fail("Should have matched", id, indexedShapes, indexedShapesGS, queryShape);
+                }
+            }
+        }
+
+        private IShape randomShapePairRect(bool biasContains)
+        {
+            IRectangle shape1 = randomRectangle();
+            IRectangle shape2 = randomRectangle();
+            return new ShapePair(shape1, shape2, biasContains, ctx);
+        }
+
+        private void fail(String label, String id, IDictionary<String, IShape> indexedShapes, IDictionary<String, IShape> indexedShapesGS, IShape queryShape)
+        {
+            Console.WriteLine("Ig:" + indexedShapesGS[id] + " Qg:" + gridSnap(queryShape));
+            fail(label + " I#" + id + ":" + indexedShapes[id] + " Q:" + queryShape);
+        }
+
+        //  private Rectangle inset(Rectangle r) {
+        //    //typically inset by 1 (whole numbers are easy to read)
+        //    double d = Math.min(1.0, grid.getDistanceForLevel(grid.getMaxLevels()) / 4);
+        //    return ctx.makeRectangle(r.getMinX() + d, r.getMaxX() - d, r.getMinY() + d, r.getMaxY() - d);
+        //  }
+
+        protected IShape gridSnap(IShape snapMe)
+        {
+            if (snapMe == null)
+                return null;
+            if (snapMe is ShapePair)
+            {
+                ShapePair me = (ShapePair)snapMe;
+                return new ShapePair(gridSnap(me.shape1), gridSnap(me.shape2), me.biasContainsThenWithin, ctx);
+            }
+            if (snapMe is IPoint)
+            {
+                snapMe = snapMe.BoundingBox;
+            }
+            //The next 4 lines mimic PrefixTreeStrategy.createIndexableFields()
+            double distErrPct = ((PrefixTreeStrategy)strategy).DistErrPct;
+            double distErr = SpatialArgs.CalcDistanceFromErrPct(snapMe, distErrPct, ctx);
+            int detailLevel = grid.GetLevelForDistance(distErr);
+            IList<Cell> cells = grid.GetCells(snapMe, detailLevel, false, true);
+
+            //calc bounding box of cells.
+            List<IShape> cellShapes = new List<IShape>(cells.size());
+            foreach (Cell cell in cells)
+            {
+                cellShapes.Add(cell.GetShape());
+            }
+            return new ShapeCollection(cellShapes, ctx).BoundingBox;
+        }
+
+        /**
+         * An aggregate of 2 shapes. Unfortunately we can't simply use a ShapeCollection because:
+         * (a) ambiguity between CONTAINS & WITHIN for equal shapes, and
+         * (b) adjacent pairs could as a whole contain the input shape.
+         * The tests here are sensitive to these matters, although in practice ShapeCollection
+         * is fine.
+         */
+        private class ShapePair : ShapeCollection /*<Shape>*/
+        {
+
+            private readonly SpatialContext ctx;
+            internal IShape shape1, shape2;
+            internal bool biasContainsThenWithin;//a hack
+
+            public ShapePair(IShape shape1, IShape shape2, bool containsThenWithin, SpatialContext ctx)
+                        : base(Arrays.AsList(shape1, shape2), ctx)
+            {
+
+                this.shape1 = shape1;
+                this.shape2 = shape2;
+                biasContainsThenWithin = containsThenWithin;
+            }
+
+            public override SpatialRelation Relate(IShape other)
+            {
+                SpatialRelation r = relateApprox(other);
+                if (r == SpatialRelation.CONTAINS)
+                    return r;
+                if (r == SpatialRelation.DISJOINT)
+                    return r;
+                if (r == SpatialRelation.WITHIN && !biasContainsThenWithin)
+                    return r;
+
+                //See if the correct answer is actually Contains, when the indexed shapes are adjacent,
+                // creating a larger shape that contains the input shape.
+                bool pairTouches = shape1.Relate(shape2).Intersects();
+                if (!pairTouches)
+                    return r;
+                //test all 4 corners
+                IRectangle oRect = (IRectangle)other;
+                if (Relate(ctx.MakePoint(oRect.MinX, oRect.MinY)) == SpatialRelation.CONTAINS
+                    && Relate(ctx.MakePoint(oRect.MinX, oRect.MaxY)) == SpatialRelation.CONTAINS
+                    && Relate(ctx.MakePoint(oRect.MaxX, oRect.MinY)) == SpatialRelation.CONTAINS
+                    && Relate(ctx.MakePoint(oRect.MaxX, oRect.MaxY)) == SpatialRelation.CONTAINS)
+                    return SpatialRelation.CONTAINS;
+                return r;
+            }
+
+            private SpatialRelation relateApprox(IShape other)
+            {
+                if (biasContainsThenWithin)
+                {
+                    if (shape1.Relate(other) == SpatialRelation.CONTAINS || shape1.equals(other)
+                        || shape2.Relate(other) == SpatialRelation.CONTAINS || shape2.equals(other)) return SpatialRelation.CONTAINS;
+
+                    if (shape1.Relate(other) == SpatialRelation.WITHIN && shape2.Relate(other) == SpatialRelation.WITHIN) return SpatialRelation.WITHIN;
+
+                }
+                else
+                {
+                    if ((shape1.Relate(other) == SpatialRelation.WITHIN || shape1.equals(other))
+                        && (shape2.Relate(other) == SpatialRelation.WITHIN || shape2.equals(other))) return SpatialRelation.WITHIN;
+
+                    if (shape1.Relate(other) == SpatialRelation.CONTAINS || shape2.Relate(other) == SpatialRelation.CONTAINS) return SpatialRelation.CONTAINS;
+                }
+
+                if (shape1.Relate(other).Intersects() || shape2.Relate(other).Intersects())
+                    return SpatialRelation.INTERSECTS;//might actually be 'CONTAINS' if the pair are adjacent but we handle that later
+                return SpatialRelation.DISJOINT;
+            }
+
+            public override String ToString()
+            {
+                return "ShapePair(" + shape1 + " , " + shape2 + ")";
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Prefix/TestRecursivePrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Prefix/TestRecursivePrefixTreeStrategy.cs b/src/Lucene.Net.Tests.Spatial/Prefix/TestRecursivePrefixTreeStrategy.cs
new file mode 100644
index 0000000..f55b189
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Prefix/TestRecursivePrefixTreeStrategy.cs
@@ -0,0 +1,127 @@
+\ufeffusing Lucene.Net.Spatial.Prefix.Tree;
+using Lucene.Net.Spatial.Queries;
+using NUnit.Framework;
+using Spatial4n.Core.Context;
+using Spatial4n.Core.Distance;
+using Spatial4n.Core.Shapes;
+using System.Collections.Generic;
+using System.Globalization;
+
+namespace Lucene.Net.Spatial.Prefix
+{
+    /*
+     * 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.
+     */
+
+    public class TestRecursivePrefixTreeStrategy : StrategyTestCase
+    {
+        private int maxLength;
+
+        //Tests should call this first.
+        private void init(int maxLength)
+        {
+            this.maxLength = maxLength;
+            this.ctx = SpatialContext.GEO;
+            GeohashPrefixTree grid = new GeohashPrefixTree(ctx, maxLength);
+            this.strategy = new RecursivePrefixTreeStrategy(grid, GetType().Name);
+        }
+
+        [Test]
+        public virtual void TestFilterWithVariableScanLevel()
+        {
+            init(GeohashPrefixTree.MaxLevelsPossible);
+            getAddAndVerifyIndexedDocuments(DATA_WORLD_CITIES_POINTS);
+
+            //execute queries for each prefix grid scan level
+            for (int i = 0; i <= maxLength; i++)
+            {
+                ((RecursivePrefixTreeStrategy)strategy).PrefixGridScanLevel = (i);
+                executeQueries(SpatialMatchConcern.FILTER, QTEST_Cities_Intersects_BBox);
+            }
+        }
+
+        [Test]
+        public virtual void TestOneMeterPrecision()
+        {
+            init(GeohashPrefixTree.MaxLevelsPossible);
+            GeohashPrefixTree grid = (GeohashPrefixTree)((RecursivePrefixTreeStrategy)strategy).Grid;
+            //DWS: I know this to be true.  11 is needed for one meter
+            double degrees = DistanceUtils.Dist2Degrees(0.001, DistanceUtils.EARTH_MEAN_RADIUS_KM);
+            assertEquals(11, grid.GetLevelForDistance(degrees));
+        }
+
+        [Test]
+        public virtual void TestPrecision()
+        {
+            init(GeohashPrefixTree.MaxLevelsPossible);
+
+            Spatial4n.Core.Shapes.IPoint iPt = ctx.MakePoint(2.8028712999999925, 48.3708044);//lon, lat
+            AddDocument(newDoc("iPt", iPt));
+            Commit();
+
+            Spatial4n.Core.Shapes.IPoint qPt = ctx.MakePoint(2.4632387000000335, 48.6003516);
+
+            double KM2DEG = DistanceUtils.Dist2Degrees(1, DistanceUtils.EARTH_MEAN_RADIUS_KM);
+            double DEG2KM = 1 / KM2DEG;
+
+            double DIST = 35.75;//35.7499...
+            assertEquals(DIST, ctx.DistCalc.Distance(iPt, qPt) * DEG2KM, 0.001);
+
+            //distErrPct will affect the query shape precision. The indexed precision
+            // was set to nearly zilch via init(GeohashPrefixTree.getMaxLevelsPossible());
+            double distErrPct = 0.025; //the suggested default, by the way
+            double distMult = 1 + distErrPct;
+
+            assertTrue(35.74 * distMult >= DIST);
+            checkHits(q(qPt, 35.74 * KM2DEG, distErrPct), 1, null);
+
+            assertTrue(30 * distMult < DIST);
+            checkHits(q(qPt, 30 * KM2DEG, distErrPct), 0, null);
+
+            assertTrue(33 * distMult < DIST);
+            checkHits(q(qPt, 33 * KM2DEG, distErrPct), 0, null);
+
+            assertTrue(34 * distMult < DIST);
+            checkHits(q(qPt, 34 * KM2DEG, distErrPct), 0, null);
+        }
+
+        private SpatialArgs q(Spatial4n.Core.Shapes.IPoint pt, double distDEG, double distErrPct)
+        {
+            IShape shape = ctx.MakeCircle(pt, distDEG);
+            SpatialArgs args = new SpatialArgs(SpatialOperation.Intersects, shape);
+            args.DistErrPct = (distErrPct);
+            return args;
+        }
+
+        private void checkHits(SpatialArgs args, int assertNumFound, int[] assertIds)
+        {
+            SearchResults got = executeQuery(strategy.MakeQuery(args), 100);
+            assertEquals("" + args, assertNumFound, got.numFound);
+            if (assertIds != null)
+            {
+                ISet<int?> gotIds = new HashSet<int?>();
+                foreach (SearchResult result in got.results)
+                {
+                    gotIds.add(int.Parse(result.document.Get("id"), CultureInfo.InvariantCulture));
+                }
+                foreach (int assertId in assertIds)
+                {
+                    assertTrue("has " + assertId, gotIds.contains(assertId));
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Prefix/TestTermQueryPrefixGridStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Prefix/TestTermQueryPrefixGridStrategy.cs b/src/Lucene.Net.Tests.Spatial/Prefix/TestTermQueryPrefixGridStrategy.cs
new file mode 100644
index 0000000..02afd84
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Prefix/TestTermQueryPrefixGridStrategy.cs
@@ -0,0 +1,60 @@
+\ufeffusing Lucene.Net.Documents;
+using Lucene.Net.Index;
+using Lucene.Net.Spatial.Prefix.Tree;
+using Lucene.Net.Spatial.Queries;
+using Lucene.Net.Support;
+using NUnit.Framework;
+using Spatial4n.Core.Context;
+
+namespace Lucene.Net.Spatial.Prefix
+{
+    /*
+     * 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.
+     */
+
+    public class TestTermQueryPrefixGridStrategy : SpatialTestCase
+    {
+        [Test]
+        public virtual void TestNGramPrefixGridLosAngeles()
+        {
+            SpatialContext ctx = SpatialContext.GEO;
+            TermQueryPrefixTreeStrategy prefixGridStrategy = new TermQueryPrefixTreeStrategy(new QuadPrefixTree(ctx), "geo");
+
+            Spatial4n.Core.Shapes.IShape point = ctx.MakePoint(-118.243680, 34.052230);
+
+            Document losAngeles = new Document();
+            losAngeles.Add(new StringField("name", "Los Angeles", Field.Store.YES));
+            foreach (IndexableField field in prefixGridStrategy.CreateIndexableFields(point))
+            {
+                losAngeles.Add(field);
+            }
+            losAngeles.Add(new StoredField(prefixGridStrategy.FieldName, point.toString()));//just for diagnostics
+
+            addDocumentsAndCommit(Arrays.AsList(losAngeles));
+
+            // This won't work with simple spatial context...
+            SpatialArgsParser spatialArgsParser = new SpatialArgsParser();
+            // TODO... use a non polygon query
+            //    SpatialArgs spatialArgs = spatialArgsParser.parse(
+            //        "Intersects(POLYGON((-127.00390625 39.8125,-112.765625 39.98828125,-111.53515625 31.375,-125.94921875 30.14453125,-127.00390625 39.8125)))",
+            //        new SimpleSpatialContext());
+
+            //    Query query = prefixGridStrategy.makeQuery(spatialArgs, fieldInfo);
+            //    SearchResults searchResults = executeQuery(query, 1);
+            //    assertEquals(1, searchResults.numFound);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Prefix/Tree/SpatialPrefixTreeTest.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Prefix/Tree/SpatialPrefixTreeTest.cs b/src/Lucene.Net.Tests.Spatial/Prefix/Tree/SpatialPrefixTreeTest.cs
new file mode 100644
index 0000000..581b2f6
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Prefix/Tree/SpatialPrefixTreeTest.cs
@@ -0,0 +1,105 @@
+\ufeffusing Lucene.Net.Documents;
+using Lucene.Net.Search;
+using Lucene.Net.Spatial.Queries;
+using NUnit.Framework;
+using Spatial4n.Core.Context;
+using Spatial4n.Core.Shapes;
+using System;
+
+namespace Lucene.Net.Spatial.Prefix.Tree
+{
+    /*
+     * 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.
+     */
+
+    public class SpatialPrefixTreeTest : SpatialTestCase
+    {
+        //TODO plug in others and test them
+        new private SpatialContext ctx;
+        private SpatialPrefixTree trie;
+
+        public override void SetUp()
+        {
+            base.SetUp();
+            ctx = SpatialContext.GEO;
+        }
+
+        [Test]
+        public virtual void TestCellTraverse()
+        {
+            trie = new GeohashPrefixTree(ctx, 4);
+
+            Cell prevC = null;
+            Cell c = trie.WorldCell;
+            assertEquals(0, c.Level);
+            assertEquals(ctx.WorldBounds, c.GetShape());
+            while (c.Level < trie.MaxLevels)
+            {
+                prevC = c;
+
+                var iter = c.GetSubCells().GetEnumerator();
+                iter.MoveNext();
+                c = iter.Current;
+                //c = c.GetSubCells().GetEnumerator().next();//TODO random which one?
+
+                assertEquals(prevC.Level + 1, c.Level);
+                IRectangle prevNShape = (IRectangle)prevC.GetShape();
+                IShape s = c.GetShape();
+                IRectangle sbox = s.BoundingBox;
+                assertTrue(prevNShape.Width > sbox.Width);
+                assertTrue(prevNShape.Height > sbox.Height);
+            }
+        }
+        /**
+         * A PrefixTree pruning optimization gone bad.
+         * See <a href="https://issues.apache.org/jira/browse/LUCENE-4770>LUCENE-4770</a>.
+         */
+        [Test]
+        public virtual void TestBadPrefixTreePrune()
+        {
+
+            trie = new QuadPrefixTree(ctx, 12);
+            TermQueryPrefixTreeStrategy strategy = new TermQueryPrefixTreeStrategy(trie, "geo");
+            Document doc = new Document();
+            doc.Add(new TextField("id", "1", Field.Store.YES));
+
+            IShape area = ctx.MakeRectangle(-122.82, -122.78, 48.54, 48.56);
+
+            Field[] fields = strategy.CreateIndexableFields(area, 0.025);
+            foreach (Field field in fields)
+            {
+                doc.Add(field);
+            }
+            AddDocument(doc);
+
+            IPoint upperleft = ctx.MakePoint(-122.88, 48.54);
+            IPoint lowerright = ctx.MakePoint(-122.82, 48.62);
+
+            Query query = strategy.MakeQuery(new SpatialArgs(SpatialOperation.Intersects, ctx.MakeRectangle(upperleft, lowerright)));
+
+            Commit();
+
+            TopDocs search = indexSearcher.Search(query, 10);
+            ScoreDoc[] scoreDocs = search.ScoreDocs;
+            foreach (ScoreDoc scoreDoc in scoreDocs)
+            {
+                Console.WriteLine(indexSearcher.Doc(scoreDoc.Doc));
+            }
+
+            assertEquals(1, search.TotalHits);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Properties/AssemblyInfo.cs b/src/Lucene.Net.Tests.Spatial/Properties/AssemblyInfo.cs
new file mode 100644
index 0000000..3e1f0a5
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Properties/AssemblyInfo.cs
@@ -0,0 +1,36 @@
+\ufeffusing System.Reflection;
+using System.Runtime.CompilerServices;
+using System.Runtime.InteropServices;
+
+// General Information about an assembly is controlled through the following 
+// set of attributes. Change these attribute values to modify the information
+// associated with an assembly.
+[assembly: AssemblyTitle("Lucene.Net.Tests.Spatial")]
+[assembly: AssemblyDescription("")]
+[assembly: AssemblyConfiguration("")]
+[assembly: AssemblyCompany("")]
+[assembly: AssemblyProduct("Lucene.Net.Tests.Spatial")]
+[assembly: AssemblyCopyright("Copyright �  2016")]
+[assembly: AssemblyTrademark("")]
+[assembly: AssemblyCulture("")]
+
+// Setting ComVisible to false makes the types in this assembly not visible 
+// to COM components.  If you need to access a type in this assembly from 
+// COM, set the ComVisible attribute to true on that type.
+[assembly: ComVisible(false)]
+
+// The following GUID is for the ID of the typelib if this project is exposed to COM
+[assembly: Guid("31f52f5c-a08f-4363-8003-23d6f7d6eb3a")]
+
+// Version information for an assembly consists of the following four values:
+//
+//      Major Version
+//      Minor Version 
+//      Build Number
+//      Revision
+//
+// You can specify all the values or you can default the Build and Revision Numbers 
+// by using the '*' as shown below:
+// [assembly: AssemblyVersion("1.0.*")]
+[assembly: AssemblyVersion("1.0.0.0")]
+[assembly: AssemblyFileVersion("1.0.0.0")]

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Query/SpatialArgsParserTest.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Query/SpatialArgsParserTest.cs b/src/Lucene.Net.Tests.Spatial/Query/SpatialArgsParserTest.cs
new file mode 100644
index 0000000..ee2f9a2
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Query/SpatialArgsParserTest.cs
@@ -0,0 +1,69 @@
+\ufeffusing Lucene.Net.Util;
+using NUnit.Framework;
+using Spatial4n.Core.Context;
+using Spatial4n.Core.Shapes;
+using System;
+
+namespace Lucene.Net.Spatial.Queries
+{
+    /*
+     * 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.
+     */
+
+    public class SpatialArgsParserTest : LuceneTestCase
+    {
+        private SpatialContext ctx = SpatialContext.GEO;
+
+        //The args parser is only dependent on the ctx for IO so I don't care to test
+        // with other implementations.
+
+        [Test]
+        public virtual void TestArgsParser()
+        {
+            SpatialArgsParser parser = new SpatialArgsParser();
+
+            String arg = SpatialOperation.IsWithin + "(Envelope(-10, 10, 20, -20))";
+            SpatialArgs @out = parser.Parse(arg, ctx);
+            assertEquals(SpatialOperation.IsWithin, @out.Operation);
+            IRectangle bounds = (IRectangle)@out.Shape;
+            assertEquals(-10.0, bounds.MinX, 0D);
+            assertEquals(10.0, bounds.MaxX, 0D);
+
+            // Disjoint should not be scored
+            arg = SpatialOperation.IsDisjointTo + " (Envelope(-10,-20,20,10))";
+            @out = parser.Parse(arg, ctx);
+            assertEquals(SpatialOperation.IsDisjointTo, @out.Operation);
+
+            try
+            {
+                parser.Parse(SpatialOperation.IsDisjointTo + "[ ]", ctx);
+                fail("spatial operations need args");
+            }
+            catch (Exception ex)
+            {//expected
+            }
+
+            try
+            {
+                parser.Parse("XXXX(Envelope(-10, 10, 20, -20))", ctx);
+                fail("unknown operation!");
+            }
+            catch (Exception ex)
+            {//expected
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/QueryEqualsHashCodeTest.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/QueryEqualsHashCodeTest.cs b/src/Lucene.Net.Tests.Spatial/QueryEqualsHashCodeTest.cs
new file mode 100644
index 0000000..25db926
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/QueryEqualsHashCodeTest.cs
@@ -0,0 +1,169 @@
+\ufeffusing Lucene.Net.Spatial.Prefix;
+using Lucene.Net.Spatial.Prefix.Tree;
+using Lucene.Net.Spatial.Queries;
+using Lucene.Net.Spatial.Serialized;
+using Lucene.Net.Spatial.Vector;
+using Lucene.Net.Util;
+using NUnit.Framework;
+using Spatial4n.Core.Context;
+using Spatial4n.Core.Shapes;
+using System;
+using System.Collections.Generic;
+
+namespace Lucene.Net.Spatial
+{
+    /*
+     * 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.
+     */
+
+    public class QueryEqualsHashCodeTest : LuceneTestCase
+    {
+        private readonly SpatialContext ctx = SpatialContext.GEO;
+
+        [Test]
+        public virtual void TestEqualsHashCode()
+        {
+
+            SpatialPrefixTree gridQuad = new QuadPrefixTree(ctx, 10);
+            SpatialPrefixTree gridGeohash = new GeohashPrefixTree(ctx, 10);
+
+            List<SpatialStrategy> strategies = new List<SpatialStrategy>();
+            strategies.Add(new RecursivePrefixTreeStrategy(gridGeohash, "recursive_geohash"));
+            strategies.Add(new TermQueryPrefixTreeStrategy(gridQuad, "termquery_quad"));
+            strategies.Add(new PointVectorStrategy(ctx, "pointvector"));
+            //strategies.Add(new BBoxStrategy(ctx, "bbox"));
+            strategies.Add(new SerializedDVStrategy(ctx, "serialized"));
+            foreach (SpatialStrategy strategy in strategies)
+            {
+                TestEqualsHashcode(strategy);
+            }
+        }
+
+        private class ObjGeneratorQueryAnonymousHelper : ObjGenerator
+        {
+            private readonly SpatialStrategy strategy;
+
+            public ObjGeneratorQueryAnonymousHelper(SpatialStrategy strategy)
+            {
+                this.strategy = strategy;
+            }
+
+            public object gen(SpatialArgs args)
+            {
+                return strategy.MakeQuery(args);
+            }
+        }
+
+        private class ObjGeneratorFilterAnonymousHelper : ObjGenerator
+        {
+            private readonly SpatialStrategy strategy;
+
+            public ObjGeneratorFilterAnonymousHelper(SpatialStrategy strategy)
+            {
+                this.strategy = strategy;
+            }
+
+            public object gen(SpatialArgs args)
+            {
+                return strategy.MakeFilter(args);
+            }
+        }
+
+        private class ObjGeneratorDistanceValueSourceAnonymousHelper : ObjGenerator
+        {
+            private readonly SpatialStrategy strategy;
+
+            public ObjGeneratorDistanceValueSourceAnonymousHelper(SpatialStrategy strategy)
+            {
+                this.strategy = strategy;
+            }
+
+            public object gen(SpatialArgs args)
+            {
+                return strategy.MakeDistanceValueSource(args.Shape.Center);
+            }
+        }
+
+        private void TestEqualsHashcode(SpatialStrategy strategy)
+        {
+            SpatialArgs args1 = MakeArgs1();
+            SpatialArgs args2 = MakeArgs2();
+            TestEqualsHashcode(args1, args2, new ObjGeneratorQueryAnonymousHelper(strategy));
+            //        testEqualsHashcode(args1, args2, new ObjGenerator() {
+            //  @Override
+            //  public Object gen(SpatialArgs args)
+            //    {
+            //        return strategy.makeQuery(args);
+            //    }
+            //});
+            TestEqualsHashcode(args1, args2, new ObjGeneratorFilterAnonymousHelper(strategy));
+            //    testEqualsHashcode(args1, args2, new ObjGenerator()
+            //    {
+            //        @Override
+            //      public Object gen(SpatialArgs args)
+            //    {
+            //        return strategy.makeFilter(args);
+            //    }
+            //});
+            TestEqualsHashcode(args1, args2, new ObjGeneratorDistanceValueSourceAnonymousHelper(strategy));
+            //            testEqualsHashcode(args1, args2, new ObjGenerator()
+            //{
+            //    @Override
+            //      public Object gen(SpatialArgs args)
+            //{
+            //    return strategy.makeDistanceValueSource(args.getShape().getCenter());
+            //}
+            //    });
+        }
+
+        private void TestEqualsHashcode(SpatialArgs args1, SpatialArgs args2, ObjGenerator generator)
+        {
+            Object first;
+            try
+            {
+                first = generator.gen(args1);
+            }
+            catch (NotSupportedException e)
+            {
+                return;
+            }
+            if (first == null)
+                return;//unsupported op?
+            Object second = generator.gen(args1);//should be the same
+            assertEquals(first, second);
+            assertEquals(first.GetHashCode(), second.GetHashCode());
+            second = generator.gen(args2);//now should be different
+            assertNotSame(args1, args2);
+        }
+
+        private SpatialArgs MakeArgs1()
+        {
+            IShape shape1 = ctx.MakeRectangle(0, 0, 10, 10);
+            return new SpatialArgs(SpatialOperation.Intersects, shape1);
+        }
+
+        private SpatialArgs MakeArgs2()
+        {
+            IShape shape2 = ctx.MakeRectangle(0, 0, 20, 20);
+            return new SpatialArgs(SpatialOperation.Intersects, shape2);
+        }
+
+        interface ObjGenerator
+        {
+            Object gen(SpatialArgs args);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Serialized/SerializedStrategyTest.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Serialized/SerializedStrategyTest.cs b/src/Lucene.Net.Tests.Spatial/Serialized/SerializedStrategyTest.cs
new file mode 100644
index 0000000..b4d00a6
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Serialized/SerializedStrategyTest.cs
@@ -0,0 +1,73 @@
+\ufeffusing Lucene.Net.Search;
+using NUnit.Framework;
+using Spatial4n.Core.Context;
+
+namespace Lucene.Net.Spatial.Serialized
+{
+    /*
+     * 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.
+     */
+
+    public class SerializedStrategyTest : StrategyTestCase
+    {
+        public override void SetUp()
+        {
+            base.SetUp();
+            this.ctx = SpatialContext.GEO;
+            this.strategy = new SerializedDVStrategy(ctx, "serialized");
+        }
+
+
+        protected override bool NeedsDocValues()
+        {
+            return true;
+        }
+
+        //called by StrategyTestCase; we can't let it call our makeQuery which will UOE ex.
+        protected override Query makeQuery(SpatialTestQuery q)
+        {
+            return new FilteredQuery(new MatchAllDocsQuery(), strategy.MakeFilter(q.args),
+                FilteredQuery.QUERY_FIRST_FILTER_STRATEGY);
+        }
+
+        [Test]
+        public virtual void TestBasicOperaions()
+        {
+            getAddAndVerifyIndexedDocuments(DATA_SIMPLE_BBOX);
+
+            executeQueries(SpatialMatchConcern.EXACT, QTEST_Simple_Queries_BBox);
+        }
+
+        [Test]
+        public virtual void TestStatesBBox()
+        {
+            getAddAndVerifyIndexedDocuments(DATA_STATES_BBOX);
+
+            executeQueries(SpatialMatchConcern.FILTER, QTEST_States_IsWithin_BBox);
+            executeQueries(SpatialMatchConcern.FILTER, QTEST_States_Intersects_BBox);
+        }
+
+        [Test]
+        public virtual void TestCitiesIntersectsBBox()
+        {
+            getAddAndVerifyIndexedDocuments(DATA_WORLD_CITIES_POINTS);
+
+            executeQueries(SpatialMatchConcern.FILTER, QTEST_Cities_Intersects_BBox);
+        }
+
+        //sorting is tested in DistanceStrategyTest
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/SpatialArgsTest.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/SpatialArgsTest.cs b/src/Lucene.Net.Tests.Spatial/SpatialArgsTest.cs
new file mode 100644
index 0000000..afac670
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/SpatialArgsTest.cs
@@ -0,0 +1,52 @@
+\ufeffusing Lucene.Net.Spatial.Queries;
+using Lucene.Net.Util;
+using NUnit.Framework;
+using Spatial4n.Core.Context;
+using Spatial4n.Core.Shapes;
+
+namespace Lucene.Net.Spatial
+{
+    /*
+     * 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.
+     */
+
+    public class SpatialArgsTest : LuceneTestCase
+    {
+        [Test]
+        public void CalcDistanceFromErrPct()
+        {
+            SpatialContext ctx = SpatialContext.GEO;
+            double DEP = 0.5;//distErrPct
+
+            //the result is the diagonal distance from the center to the closest corner,
+            // times distErrPct
+
+            IShape superwide = ctx.MakeRectangle(-180, 180, 0, 0);
+            //0 distErrPct means 0 distance always
+            assertEquals(0, SpatialArgs.CalcDistanceFromErrPct(superwide, 0, ctx), 0);
+            assertEquals(180 * DEP, SpatialArgs.CalcDistanceFromErrPct(superwide, DEP, ctx), 0);
+
+            IShape supertall = ctx.MakeRectangle(0, 0, -90, 90);
+            assertEquals(90 * DEP, SpatialArgs.CalcDistanceFromErrPct(supertall, DEP, ctx), 0);
+
+            IShape upperhalf = ctx.MakeRectangle(-180, 180, 0, 90);
+            assertEquals(45 * DEP, SpatialArgs.CalcDistanceFromErrPct(upperhalf, DEP, ctx), 0.0001);
+
+            IShape midCircle = ctx.MakeCircle(0, 0, 45);
+            assertEquals(60 * DEP, SpatialArgs.CalcDistanceFromErrPct(midCircle, DEP, ctx), 0.0001);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/SpatialExample.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/SpatialExample.cs b/src/Lucene.Net.Tests.Spatial/SpatialExample.cs
new file mode 100644
index 0000000..7e0f7f7
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/SpatialExample.cs
@@ -0,0 +1,196 @@
+\ufeffusing Lucene.Net.Documents;
+using Lucene.Net.Index;
+using Lucene.Net.Queries.Function;
+using Lucene.Net.Search;
+using Lucene.Net.Spatial.Prefix;
+using Lucene.Net.Spatial.Prefix.Tree;
+using Lucene.Net.Spatial.Queries;
+using Lucene.Net.Store;
+using Lucene.Net.Util;
+using NUnit.Framework;
+using Spatial4n.Core.Context;
+using Spatial4n.Core.Distance;
+using Spatial4n.Core.Shapes;
+using System;
+using System.Globalization;
+
+namespace Lucene.Net.Spatial
+{
+    /*
+     * 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.
+     */
+
+    /// <summary>
+    /// This class serves as example code to show how to use the Lucene spatial
+    /// module.
+    /// </summary>
+    public class SpatialExample : LuceneTestCase
+    {
+        //Note: Test invoked via TestTestFramework.spatialExample()
+        public static void Main(String[] args)
+        {
+            new SpatialExample().Test();
+        }
+
+        [Test]
+        public virtual void Test()
+        {
+            Init();
+            IndexPoints();
+            Search();
+        }
+
+        /**
+         * The Spatial4j {@link SpatialContext} is a sort of global-ish singleton
+         * needed by Lucene spatial.  It's a facade to the rest of Spatial4j, acting
+         * as a factory for {@link Shape}s and provides access to reading and writing
+         * them from Strings.
+         */
+        private SpatialContext ctx;//"ctx" is the conventional variable name
+
+        /**
+         * The Lucene spatial {@link SpatialStrategy} encapsulates an approach to
+         * indexing and searching shapes, and providing distance values for them.
+         * It's a simple API to unify different approaches. You might use more than
+         * one strategy for a shape as each strategy has its strengths and weaknesses.
+         * <p />
+         * Note that these are initialized with a field name.
+         */
+        private SpatialStrategy strategy;
+
+        private Directory directory;
+
+        protected void Init()
+        {
+            //Typical geospatial context
+            //  These can also be constructed from SpatialContextFactory
+            this.ctx = SpatialContext.GEO;
+
+            int maxLevels = 11;//results in sub-meter precision for geohash
+                               //TODO demo lookup by detail distance
+                               //  This can also be constructed from SpatialPrefixTreeFactory
+            SpatialPrefixTree grid = new GeohashPrefixTree(ctx, maxLevels);
+
+            this.strategy = new RecursivePrefixTreeStrategy(grid, "myGeoField");
+
+            this.directory = new RAMDirectory();
+        }
+
+        private void IndexPoints()
+        {
+            IndexWriterConfig iwConfig = new IndexWriterConfig(TEST_VERSION_CURRENT, null);
+            IndexWriter indexWriter = new IndexWriter(directory, iwConfig);
+
+            //Spatial4j is x-y order for arguments
+            indexWriter.AddDocument(NewSampleDocument(
+                2, ctx.MakePoint(-80.93, 33.77)));
+
+            //Spatial4j has a WKT parser which is also "x y" order
+            indexWriter.AddDocument(NewSampleDocument(
+                4, ctx.ReadShapeFromWkt("POINT(60.9289094 -50.7693246)")));
+
+            indexWriter.AddDocument(NewSampleDocument(
+                20, ctx.MakePoint(0.1, 0.1), ctx.MakePoint(0, 0)));
+
+            indexWriter.Dispose();
+        }
+
+        private Document NewSampleDocument(int id, params IShape[] shapes)
+        {
+            Document doc = new Document();
+            doc.Add(new IntField("id", id, Field.Store.YES));
+            //Potentially more than one shape in this field is supported by some
+            // strategies; see the javadocs of the SpatialStrategy impl to see.
+            foreach (IShape shape in shapes)
+            {
+                foreach (IndexableField f in strategy.CreateIndexableFields(shape))
+                {
+                    doc.Add(f);
+                }
+                //store it too; the format is up to you
+                //  (assume point in this example)
+                IPoint pt = (IPoint)shape;
+                doc.Add(new StoredField(strategy.FieldName, pt.X + " " + pt.Y));
+            }
+
+            return doc;
+        }
+
+        private void Search()
+        {
+            IndexReader indexReader = DirectoryReader.Open(directory);
+            IndexSearcher indexSearcher = new IndexSearcher(indexReader);
+            Sort idSort = new Sort(new SortField("id", SortField.Type_e.INT));
+
+            //--Filter by circle (<= distance from a point)
+            {
+                //Search with circle
+                //note: SpatialArgs can be parsed from a string
+                SpatialArgs args = new SpatialArgs(SpatialOperation.Intersects,
+                    ctx.MakeCircle(-80.0, 33.0, DistanceUtils.Dist2Degrees(200, DistanceUtils.EARTH_MEAN_RADIUS_KM)));
+                Filter filter = strategy.MakeFilter(args);
+                TopDocs docs = indexSearcher.Search(new MatchAllDocsQuery(), filter, 10, idSort);
+                AssertDocMatchedIds(indexSearcher, docs, 2);
+                //Now, lets get the distance for the 1st doc via computing from stored point value:
+                // (this computation is usually not redundant)
+                Document doc1 = indexSearcher.Doc(docs.ScoreDocs[0].Doc);
+                String doc1Str = doc1.GetField(strategy.FieldName).StringValue;
+                //assume doc1Str is "x y" as written in newSampleDocument()
+                int spaceIdx = doc1Str.IndexOf(' ');
+                double x = double.Parse(doc1Str.Substring(0, spaceIdx - 0), CultureInfo.InvariantCulture);
+                double y = double.Parse(doc1Str.Substring(spaceIdx + 1), CultureInfo.InvariantCulture);
+                double doc1DistDEG = ctx.CalcDistance(args.Shape.Center, x, y);
+                assertEquals(121.6d, DistanceUtils.Degrees2Dist(doc1DistDEG, DistanceUtils.EARTH_MEAN_RADIUS_KM), 0.1);
+                //or more simply:
+                assertEquals(121.6d, doc1DistDEG * DistanceUtils.DEG_TO_KM, 0.1);
+            }
+            //--Match all, order by distance ascending
+            {
+                IPoint pt = ctx.MakePoint(60, -50);
+                ValueSource valueSource = strategy.MakeDistanceValueSource(pt, DistanceUtils.DEG_TO_KM);//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);
+                //To get the distance, we could compute from stored values like earlier.
+                // However in this example we sorted on it, and the distance will get
+                // computed redundantly.  If the distance is only needed for the top-X
+                // search results then that's not a big deal. Alternatively, try wrapping
+                // the ValueSource with CachingDoubleValueSource then retrieve the value
+                // from the ValueSource now. See LUCENE-4541 for an example.
+            }
+            //demo arg parsing
+            {
+                SpatialArgs args = new SpatialArgs(SpatialOperation.Intersects,
+                    ctx.MakeCircle(-80.0, 33.0, 1));
+                SpatialArgs args2 = new SpatialArgsParser().Parse("Intersects(BUFFER(POINT(-80 33),1))", ctx);
+                assertEquals(args.toString(), args2.toString());
+            }
+
+            indexReader.Dispose();
+        }
+
+        private void AssertDocMatchedIds(IndexSearcher indexSearcher, TopDocs docs, params int[] ids)
+        {
+            int[]
+            gotIds = new int[docs.TotalHits];
+            for (int i = 0; i < gotIds.Length; i++)
+            {
+                gotIds[i] = Convert.ToInt32(indexSearcher.Doc(docs.ScoreDocs[i].Doc).GetField("id").NumericValue, CultureInfo.InvariantCulture);
+            }
+            assertArrayEquals(ids, gotIds);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/SpatialMatchConcern.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/SpatialMatchConcern.cs b/src/Lucene.Net.Tests.Spatial/SpatialMatchConcern.cs
new file mode 100644
index 0000000..035f279
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/SpatialMatchConcern.cs
@@ -0,0 +1,35 @@
+\ufeffnamespace Lucene.Net.Spatial
+{
+    /*
+     * 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.
+     */
+
+    public class SpatialMatchConcern
+    {
+        public readonly bool orderIsImportant;
+        public readonly bool resultsAreSuperset; // if the strategy can not give exact answers, but used to limit results
+
+        private SpatialMatchConcern(bool order, bool superset)
+        {
+            this.orderIsImportant = order;
+            this.resultsAreSuperset = superset;
+        }
+
+        public static readonly SpatialMatchConcern EXACT = new SpatialMatchConcern(true, false);
+        public static readonly SpatialMatchConcern FILTER = new SpatialMatchConcern(false, false);
+        public static readonly SpatialMatchConcern SUPERSET = new SpatialMatchConcern(false, true);
+    }
+}


[25/26] lucenenet git commit: Spatial: Updated documentation comments

Posted by ni...@apache.org.
Spatial: Updated documentation comments


Project: http://git-wip-us.apache.org/repos/asf/lucenenet/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucenenet/commit/8219d871
Tree: http://git-wip-us.apache.org/repos/asf/lucenenet/tree/8219d871
Diff: http://git-wip-us.apache.org/repos/asf/lucenenet/diff/8219d871

Branch: refs/heads/master
Commit: 8219d871ae5b76b57e6317d8ad74267c41b7e200
Parents: b680810
Author: Shad Storhaug <sh...@shadstorhaug.com>
Authored: Fri Nov 18 19:52:43 2016 +0700
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Fri Nov 25 17:58:43 2016 +0700

----------------------------------------------------------------------
 src/Lucene.Net.Spatial/DisjointSpatialFilter.cs | 16 +---
 .../Prefix/AbstractVisitingPrefixTreeFilter.cs  | 93 ++++++------------
 .../Prefix/ContainsPrefixTreeFilter.cs          | 11 +--
 .../Prefix/IntersectsPrefixTreeFilter.cs        |  7 +-
 .../Prefix/PointPrefixTreeFieldCacheProvider.cs | 13 ++-
 .../Prefix/PrefixTreeStrategy.cs                | 99 ++++++++------------
 .../Prefix/RecursivePrefixTreeStrategy.cs       | 24 ++---
 .../Prefix/TermQueryPrefixTreeStrategy.cs       | 21 ++---
 src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs      | 56 +++++------
 .../Prefix/Tree/GeohashPrefixTree.cs            | 15 +--
 .../Prefix/Tree/QuadPrefixTree.cs               | 11 +--
 .../Prefix/Tree/SpatialPrefixTree.cs            | 43 +++------
 .../Prefix/Tree/SpatialPrefixTreeFactory.cs     | 15 ++-
 .../Prefix/WithinPrefixTreeFilter.cs            | 27 ++----
 src/Lucene.Net.Spatial/Query/SpatialArgs.cs     | 16 ++--
 .../Query/SpatialArgsParser.cs                  | 10 +-
 .../Query/SpatialOperation.cs                   |  4 +-
 .../Query/UnsupportedSpatialOperation.cs        |  5 +
 .../Serialized/SerializedDVStrategy.cs          |  8 +-
 src/Lucene.Net.Spatial/SpatialStrategy.cs       | 50 +++++-----
 src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs  |  2 +-
 .../Util/ShapePredicateValueSource.cs           |  4 +-
 .../Util/ValueSourceFilter.cs                   |  2 +-
 .../Vector/DistanceValueSource.cs               |  6 +-
 .../Vector/PointVectorStrategy.cs               | 25 ++++-
 .../Prefix/NtsPolygonTest.cs                    |  2 +-
 26 files changed, 248 insertions(+), 337 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs b/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
index 509b5cf..37e7f05 100644
--- a/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
+++ b/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
@@ -25,21 +25,15 @@ namespace Lucene.Net.Spatial
      */
 
     /// <summary>
-    /// A Spatial Filter implementing
-    /// <see cref="SpatialOperation.IsDisjointTo">Org.Apache.Lucene.Spatial.Query.SpatialOperation.IsDisjointTo
-    /// 	</see>
-    /// in terms
-    /// of a
-    /// <see cref="SpatialStrategy">SpatialStrategy</see>
-    /// 's support for
-    /// <see cref="SpatialOperation.Intersects">Org.Apache.Lucene.Spatial.Query.SpatialOperation.Intersects
-    /// 	</see>
-    /// .
+    /// A Spatial Filter implementing <see cref="SpatialOperation.IsDisjointTo"/> in terms
+    /// of a <see cref="SpatialStrategy">SpatialStrategy</see>'s support for
+    /// <see cref="SpatialOperation.Intersects"/>.
     /// A document is considered disjoint if it has spatial data that does not
     /// intersect with the query shape.  Another way of looking at this is that it's
     /// a way to invert a query shape.
+    /// 
+    /// @lucene.experimental
     /// </summary>
-    /// <lucene.experimental></lucene.experimental>
     public class DisjointSpatialFilter : Filter
     {
         private readonly string field;//maybe null

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
index 6afee2c..f83ecce 100644
--- a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
@@ -31,11 +31,11 @@ namespace Lucene.Net.Spatial.Prefix
     /// Traverses a <see cref="SpatialPrefixTree">SpatialPrefixTree</see> indexed field, using the template &
     /// visitor design patterns for subclasses to guide the traversal and collect
     /// matching documents.
-    /// <p/>
-    /// Subclasses implement <see cref="Filter.GetDocIdSet(AtomicReaderContext, Bits)">Lucene.Search.Filter.GetDocIdSet(AtomicReaderContext, Bits)</see>
-    /// by instantiating a custom <see cref="VisitorTemplate">VisitorTemplate</see>
-    /// subclass (i.e. an anonymous inner class) and implement the
+    /// <para/>
+    /// Subclasses implement <see cref="Filter.GetDocIdSet(AtomicReaderContext, Bits)"/>
+    /// by instantiating a custom <see cref="VisitorTemplate"/> subclass (i.e. an anonymous inner class) and implement the
     /// required methods.
+    /// 
     /// @lucene.internal
     /// </summary>
     public abstract class AbstractVisitingPrefixTreeFilter : AbstractPrefixTreeFilter
@@ -75,42 +75,25 @@ namespace Lucene.Net.Spatial.Prefix
 
         /// <summary>
         /// An abstract class designed to make it easy to implement predicates or
-        /// other operations on a
-        /// <see cref="Lucene.Net.Spatial.Prefix.Tree.SpatialPrefixTree">Lucene.Net.Spatial.Prefix.Tree.SpatialPrefixTree
-        /// 	</see>
-        /// indexed field. An instance
+        /// other operations on a <see cref="SpatialPrefixTree"/> indexed field. An instance
         /// of this class is not designed to be re-used across AtomicReaderContext
         /// instances so simply create a new one for each call to, say a
-        /// <see cref="Lucene.Net.Search.Filter.GetDocIdSet(Lucene.Net.Index.AtomicReaderContext, Lucene.Net.Util.Bits)
-        /// 	">Lucene.Net.Search.Filter.GetDocIdSet(Lucene.Net.Index.AtomicReaderContext, Lucene.Net.Util.Bits)
-        /// 	</see>
-        /// .
-        /// The
-        /// <see cref="GetDocIdSet()">GetDocIdSet()</see>
-        /// method here starts the work. It first checks
+        /// <see cref="Lucene.Net.Search.Filter.GetDocIdSet(Lucene.Net.Index.AtomicReaderContext, Lucene.Net.Util.Bits)"/>.
+        /// The <see cref="GetDocIdSet()"/> method here starts the work. It first checks
         /// that there are indexed terms; if not it quickly returns null. Then it calls
-        /// <see cref="Start()">Start()</see>
-        /// so a subclass can set up a return value, like an
-        /// <see cref="Lucene.Net.Util.OpenBitSet">Lucene.Net.Util.OpenBitSet</see>
-        /// . Then it starts the traversal
-        /// process, calling
-        /// <see cref="FindSubCellsToVisit(Lucene.Net.Spatial.Prefix.Tree.Cell)">FindSubCellsToVisit(Lucene.Net.Spatial.Prefix.Tree.Cell)
-        /// 	</see>
-        /// which by default finds the top cells that intersect
-        /// <code>queryShape</code>
-        /// . If
+        /// <see cref="Start()">Start()</see> so a subclass can set up a return value, like an
+        /// <see cref="Lucene.Net.Util.FixedBitSet"/>. Then it starts the traversal
+        /// process, calling <see cref="FindSubCellsToVisit(Lucene.Net.Spatial.Prefix.Tree.Cell)"/>
+        /// which by default finds the top cells that intersect <c>queryShape</c>. If
         /// there isn't an indexed cell for a corresponding cell returned for this
         /// method then it's short-circuited until it finds one, at which point
-        /// <see cref="Visit(Lucene.Net.Spatial.Prefix.Tree.Cell)">Visit(Lucene.Net.Spatial.Prefix.Tree.Cell)
-        /// 	</see>
-        /// is called. At
+        /// <see cref="Visit(Lucene.Net.Spatial.Prefix.Tree.Cell)"/> is called. At
         /// some depths, of the tree, the algorithm switches to a scanning mode that
-        /// finds calls
-        /// <see cref="VisitScanned(Lucene.Net.Spatial.Prefix.Tree.Cell)">VisitScanned(Lucene.Net.Spatial.Prefix.Tree.Cell)
-        /// 	</see>
+        /// calls <see cref="VisitScanned(Lucene.Net.Spatial.Prefix.Tree.Cell)"/>
         /// for each leaf cell found.
+        /// 
+        /// @lucene.internal
         /// </summary>
-        /// <lucene.internal></lucene.internal>
         public abstract class VisitorTemplate : BaseTermsEnumTraverser
         {
             /* Future potential optimizations:
@@ -259,9 +242,7 @@ namespace Lucene.Net.Spatial.Prefix
             }
 
             /// <summary>
-            /// Called initially, and whenever
-            /// <see cref="Visit(Lucene.Net.Spatial.Prefix.Tree.Cell)">Visit(Lucene.Net.Spatial.Prefix.Tree.Cell)
-            /// 	</see>
+            /// Called initially, and whenever <see cref="Visit(Lucene.Net.Spatial.Prefix.Tree.Cell)"/>
             /// returns true.
             /// </summary>
             /// <exception cref="System.IO.IOException"></exception>
@@ -319,11 +300,7 @@ namespace Lucene.Net.Spatial.Prefix
 
             /// <summary>
             /// Called when doing a divide & conquer to find the next intersecting cells
-            /// of the query shape that are beneath
-            /// <code>cell</code>
-            /// .
-            /// <code>cell</code>
-            /// is
+            /// of the query shape that are beneath <paramref name="cell"/>. <paramref name="cell"/> is
             /// guaranteed to have an intersection and thus this must return some number
             /// of nodes.
             /// </summary>
@@ -333,16 +310,10 @@ namespace Lucene.Net.Spatial.Prefix
             }
 
             /// <summary>
-            /// Scans (
-            /// <code>termsEnum.next()</code>
-            /// ) terms until a term is found that does
+            /// Scans (<c>termsEnum.Next()</c>) terms until a term is found that does
             /// not start with curVNode's cell. If it finds a leaf cell or a cell at
-            /// level
-            /// <code>scanDetailLevel</code>
-            /// then it calls
-            /// <see cref="VisitScanned(Lucene.Net.Spatial.Prefix.Tree.Cell)">VisitScanned(Lucene.Net.Spatial.Prefix.Tree.Cell)
-            /// 	</see>
-            /// .
+            /// level <paramref name="scanDetailLevel"/> then it calls
+            /// <see cref="VisitScanned(Lucene.Net.Spatial.Prefix.Tree.Cell)"/>.
             /// </summary>
             /// <exception cref="System.IO.IOException"></exception>
             protected internal virtual void Scan(int scanDetailLevel)
@@ -370,7 +341,7 @@ namespace Lucene.Net.Spatial.Prefix
             #region Nested type: VNodeCellIterator
 
             /// <summary>
-            /// Used for <see cref="VNode.children">VNode.children</see>.
+            /// Used for <see cref="VNode.children"/>.
             /// </summary>
             private class VNodeCellIterator : IEnumerator<VNode>
             {
@@ -459,32 +430,29 @@ namespace Lucene.Net.Spatial.Prefix
 
             /// <summary>
             /// Visit an indexed cell returned from
-            /// <see cref="FindSubCellsToVisit(Lucene.Net.Spatial.Prefix.Tree.Cell)">FindSubCellsToVisit(Lucene.Net.Spatial.Prefix.Tree.Cell)
-            /// 	</see>
-            /// .
+            /// <see cref="FindSubCellsToVisit(Lucene.Net.Spatial.Prefix.Tree.Cell)"/>.
             /// </summary>
             /// <param name="cell">An intersecting cell.</param>
             /// <returns>
             /// true to descend to more levels. It is an error to return true
-            /// if cell.level == detailLevel
+            /// if cell.Level == detailLevel
             /// </returns>
             /// <exception cref="System.IO.IOException"></exception>
             protected internal abstract bool Visit(Cell cell);
 
             /// <summary>Called after visit() returns true and an indexed leaf cell is found.</summary>
             /// <remarks>
-            /// Called after visit() returns true and an indexed leaf cell is found. An
+            /// Called after Visit() returns true and an indexed leaf cell is found. An
             /// indexed leaf cell means associated documents generally won't be found at
             /// further detail levels.
             /// </remarks>
             /// <exception cref="System.IO.IOException"></exception>
             protected internal abstract void VisitLeaf(Cell cell);
 
-            /// <summary>The cell is either indexed as a leaf or is the last level of detail.</summary>
-            /// <remarks>
+            /// <summary>
             /// The cell is either indexed as a leaf or is the last level of detail. It
             /// might not even intersect the query shape, so be sure to check for that.
-            /// </remarks>
+            /// </summary>
             /// <exception cref="System.IO.IOException"></exception>
             protected internal abstract void VisitScanned(Cell cell);
 
@@ -503,13 +471,12 @@ namespace Lucene.Net.Spatial.Prefix
         #region Nested type: VNode
 
         /// <summary>
-        /// A Visitor Cell/Cell found via the query shape for
-        /// <see cref="VisitorTemplate">VisitorTemplate</see>
-        /// .
+        /// A Visitor node/cell found via the query shape for <see cref="VisitorTemplate"/>.
         /// Sometimes these are reset(cell). It's like a LinkedList node but forms a
         /// tree.
+        /// 
+        /// @lucene.internal
         /// </summary>
-        /// <lucene.internal></lucene.internal>
         public class VNode
         {
             //Note: The VNode tree adds more code to debug/maintain v.s. a flattened
@@ -521,7 +488,7 @@ namespace Lucene.Net.Spatial.Prefix
             internal IEnumerator<VNode> children;//null, then sometimes set, then null
             internal Cell cell;//not null (except initially before reset())
 
-            /// <summary>call reset(cell) after to set the cell.</summary>
+            /// <summary>Call <see cref="Reset(Cell)"/> after to set the cell.</summary>
             internal VNode(VNode parent)
             {
                 // remember to call reset(cell) after

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
index 692722e..30bcba0 100644
--- a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
@@ -27,13 +27,11 @@ namespace Lucene.Net.Spatial.Prefix
      */
 
     /// <summary>
-    /// Finds docs where its indexed shape
-    /// <see cref="SpatialOperation.Contains">CONTAINS</see>
-    /// the query shape. For use on
-    /// <see cref="RecursivePrefixTreeStrategy">RecursivePrefixTreeStrategy</see>
-    /// .
+    /// Finds docs where its indexed shape <see cref="Queries.SpatialOperation.CONTAINS"/>
+    /// the query shape. For use on <see cref="RecursivePrefixTreeStrategy"/>.
+    /// 
+    /// @lucene.experimental
     /// </summary>
-    /// <lucene.experimental></lucene.experimental>
     public class ContainsPrefixTreeFilter : AbstractPrefixTreeFilter
     {
         // Future optimizations:
@@ -247,7 +245,6 @@ namespace Lucene.Net.Spatial.Prefix
             }
 
             /// <summary>Number of docids.</summary>
-            /// <remarks>Number of docids.</remarks>
             public virtual int Size
             {
                 get { return intSet.Size(); }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
index e9916e1..c446ff2 100644
--- a/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
@@ -25,12 +25,11 @@ namespace Lucene.Net.Spatial.Prefix
      */
 
     /// <summary>
-    /// A Filter matching documents that have an
-    /// <see cref="SpatialRelation.Intersects">SpatialRelation.Intersects
-    /// 	</see>
+    /// A Filter matching documents that have an <see cref="SpatialRelation.Intersects"/>
     /// (i.e. not DISTINCT) relationship with a provided query shape.
+    /// 
+    /// @lucene.internal
     /// </summary>
-    /// <lucene.internal></lucene.internal>
     public class IntersectsPrefixTreeFilter : AbstractVisitingPrefixTreeFilter
     {
         private readonly bool hasIndexedLeaves;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs b/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
index d4a2ef3..d7cf8bd 100644
--- a/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
+++ b/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
@@ -23,15 +23,14 @@ namespace Lucene.Net.Spatial.Prefix
      */
 
     /// <summary>
-    /// Implementation of
-    /// <see cref="Lucene.Net.Spatial.Util.ShapeFieldCacheProvider{T}">Lucene.Net.Spatial.Util.ShapeFieldCacheProvider&lt;T&gt;
-    /// 	</see>
-    /// designed for
-    /// <see cref="PrefixTreeStrategy">PrefixTreeStrategy</see>s.
+    /// Implementation of <see cref="Lucene.Net.Spatial.Util.ShapeFieldCacheProvider{T}"/>
+    /// designed for <see cref="PrefixTreeStrategy">PrefixTreeStrategy</see>s.
+    /// 
     /// Note, due to the fragmented representation of Shapes in these Strategies, this implementation
-    /// can only retrieve the central <see cref="Point">Point</see> of the original Shapes.
+    /// can only retrieve the central <see cref="IPoint">Point</see> of the original Shapes.
+    /// 
+    /// @lucene.internal
     /// </summary>
-    /// <lucene.internal></lucene.internal>
     public class PointPrefixTreeFieldCacheProvider : ShapeFieldCacheProvider<IPoint>
     {
         internal readonly SpatialPrefixTree grid; //

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
index 5bd7605..cf8a130 100644
--- a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
@@ -34,59 +34,46 @@ namespace Lucene.Net.Spatial.Prefix
      */
 
     /// <summary>
-    /// An abstract SpatialStrategy based on
-    /// <see cref="Lucene.Net.Spatial.Prefix.Tree.SpatialPrefixTree">Lucene.Net.Spatial.Prefix.Tree.SpatialPrefixTree
-    /// 	</see>
-    /// . The two
-    /// subclasses are
-    /// <see cref="RecursivePrefixTreeStrategy">RecursivePrefixTreeStrategy</see>
-    /// and
-    /// <see cref="TermQueryPrefixTreeStrategy">TermQueryPrefixTreeStrategy</see>
-    /// .  This strategy is most effective as a fast
+    /// An abstract SpatialStrategy based on <see cref="Lucene.Net.Spatial.Prefix.Tree.SpatialPrefixTree"/>. The two
+    /// subclasses are <see cref="RecursivePrefixTreeStrategy">RecursivePrefixTreeStrategy</see> and
+    /// <see cref="TermQueryPrefixTreeStrategy">TermQueryPrefixTreeStrategy</see>.  This strategy is most effective as a fast
     /// approximate spatial search filter.
+    /// 
     /// <h4>Characteristics:</h4>
-    /// <ul>
-    /// <li>Can index any shape; however only
+    /// <list type="bullet">
+    /// <item>Can index any shape; however only
     /// <see cref="RecursivePrefixTreeStrategy">RecursivePrefixTreeStrategy</see>
-    /// can effectively search non-point shapes.</li>
-    /// <li>Can index a variable number of shapes per field value. This strategy
-    /// can do it via multiple calls to
-    /// <see cref="CreateIndexableFields(Shape)">CreateIndexableFields(Shape)
-    /// 	</see>
-    /// for a document or by giving it some sort of Shape aggregate (e.g. JTS
+    /// can effectively search non-point shapes.</item>
+    /// <item>Can index a variable number of shapes per field value. This strategy
+    /// can do it via multiple calls to <see cref="CreateIndexableFields(IShape)"/>
+    /// for a document or by giving it some sort of Shape aggregate (e.g. NTS
     /// WKT MultiPoint).  The shape's boundary is approximated to a grid precision.
-    /// </li>
-    /// <li>Can query with any shape.  The shape's boundary is approximated to a grid
-    /// precision.</li>
-    /// <li>Only
-    /// <see cref="Lucene.Net.Spatial.Query.SpatialOperation.Intersects">Lucene.Net.Spatial.Query.SpatialOperation.Intersects
-    /// 	</see>
+    /// </item>
+    /// <item>Can query with any shape.  The shape's boundary is approximated to a grid
+    /// precision.</item>
+    /// <item>Only <see cref="SpatialOperation.Intersects"/>
     /// is supported.  If only points are indexed then this is effectively equivalent
-    /// to IsWithin.</li>
-    /// <li>The strategy supports
-    /// <see cref="MakeDistanceValueSource(Point)">MakeDistanceValueSource(Point)
-    /// 	</see>
+    /// to IsWithin.</item>
+    /// <item>The strategy supports <see cref="MakeDistanceValueSource(IPoint, 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
+    /// behavior is undefined otherwise.  However, <c>it will likely be removed in
+    /// the future</c> in lieu of using another strategy with a more scalable
     /// implementation.  Use of this call is the only
     /// circumstance in which a cache is used.  The cache is simple but as such
     /// it doesn't scale to large numbers of points nor is it real-time-search
-    /// friendly.</li>
-    /// </ul>
+    /// friendly.</item>
+    /// </list>
+    /// 
     /// <h4>Implementation:</h4>
-    /// The
-    /// <see cref="Lucene.Net.Spatial.Prefix.Tree.SpatialPrefixTree">Lucene.Net.Spatial.Prefix.Tree.SpatialPrefixTree
-    /// 	</see>
+    /// The <see cref="Lucene.Net.Spatial.Prefix.Tree.SpatialPrefixTree"/>
     /// does most of the work, for example returning
     /// a list of terms representing grids of various sizes for a supplied shape.
     /// An important
-    /// configuration item is
-    /// <see cref="SetDistErrPct(double)">SetDistErrPct(double)</see>
-    /// which balances
-    /// shape precision against scalability.  See those javadocs.
+    /// configuration item is <see cref="SetDistErrPct(double)"/> which balances
+    /// shape precision against scalability.  See those docs.
+    /// 
+    /// @lucene.internal
     /// </summary>
-    /// <lucene.internal></lucene.internal>
     public abstract class PrefixTreeStrategy : SpatialStrategy
     {
         protected internal readonly SpatialPrefixTree grid;
@@ -106,9 +93,7 @@ namespace Lucene.Net.Spatial.Prefix
         }
 
         /// <summary>
-        /// A memory hint used by
-        /// <see cref="MakeDistanceValueSource(Point)">MakeDistanceValueSource(Point)
-        /// 	</see>
+        /// A memory hint used by <see cref="SpatialStrategy.MakeDistanceValueSource(IPoint)"/>
         /// for how big the initial size of each Document's array should be. The
         /// default is 2.  Set this to slightly more than the default expected number
         /// of points per document.
@@ -125,20 +110,13 @@ namespace Lucene.Net.Spatial.Prefix
         /// <remarks>
         /// The default measure of shape precision affecting shapes at index and query
         /// times. Points don't use this as they are always indexed at the configured
-        /// maximum precision (
-        /// <see cref="Lucene.Net.Spatial.Prefix.Tree.SpatialPrefixTree.GetMaxLevels()
-        /// 	">Lucene.Net.Spatial.Prefix.Tree.SpatialPrefixTree.GetMaxLevels()</see>
-        /// );
+        /// maximum precision (<see cref="Lucene.Net.Spatial.Prefix.Tree.SpatialPrefixTree.MaxLevels"/>);
         /// this applies to all other shapes. Specific shapes at index and query time
         /// can use something different than this default value.  If you don't set a
-        /// default then the default is
-        /// <see cref="Lucene.Net.Spatial.Query.SpatialArgs.DefaultDisterrpct">Lucene.Net.Spatial.Query.SpatialArgs.DefaultDisterrpct
-        /// 	</see>
-        /// --
+        /// default then the default is <see cref="SpatialArgs.DefaultDisterrpct"/> --
         /// 2.5%.
         /// </remarks>
-        /// <seealso cref="Lucene.Net.Spatial.Query.SpatialArgs.GetDistErrPct()">Lucene.Net.Spatial.Query.SpatialArgs.GetDistErrPct()
-        /// 	</seealso>
+        /// <seealso cref="Lucene.Net.Spatial.Queries.SpatialArgs.DistErrPct"/>
         public virtual double DistErrPct
         {
             get { return distErrPct; }
@@ -159,19 +137,22 @@ namespace Lucene.Net.Spatial.Prefix
             //TODO is CellTokenStream supposed to be re-used somehow? see Uwe's comments:
             //  http://code.google.com/p/lucene-spatial-playground/issues/detail?id=4
 
-            Field field = new Field(FieldName, new CellTokenStream(cells.GetEnumerator()), FieldType);
+            Field field = new Field(FieldName, new CellTokenStream(cells.GetEnumerator()), FIELD_TYPE);
             return new Field[] { field };
         }
 
-        public static readonly FieldType FieldType = new FieldType();
+        /// <summary>
+        /// Indexed, tokenized, not stored.
+        /// </summary>
+        public static readonly FieldType FIELD_TYPE = new FieldType();
 
         static PrefixTreeStrategy()
         {
-            FieldType.Indexed = true;
-            FieldType.Tokenized = true;
-            FieldType.OmitNorms = true;
-            FieldType.IndexOptions = FieldInfo.IndexOptions.DOCS_ONLY;
-            FieldType.Freeze();
+            FIELD_TYPE.Indexed = true;
+            FIELD_TYPE.Tokenized = true;
+            FIELD_TYPE.OmitNorms = true;
+            FIELD_TYPE.IndexOptions = FieldInfo.IndexOptions.DOCS_ONLY;
+            FIELD_TYPE.Freeze();
         }
 
         /// <summary>Outputs the tokenString of a cell, and if its a leaf, outputs it again with the leaf byte.</summary>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
index 1d49bf6..dc41b48 100644
--- a/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
@@ -23,26 +23,26 @@ namespace Lucene.Net.Spatial.Prefix
      */
 
     /// <summary>
-    /// A
-    /// <see cref="PrefixTreeStrategy">PrefixTreeStrategy</see>
-    /// which uses
-    /// <see cref="AbstractVisitingPrefixTreeFilter">AbstractVisitingPrefixTreeFilter</see>
-    /// .
+    /// A <see cref="PrefixTreeStrategy"/> which uses <see cref="AbstractVisitingPrefixTreeFilter"/>.
     /// This strategy has support for searching non-point shapes (note: not tested).
     /// Even a query shape with distErrPct=0 (fully precise to the grid) should have
     /// good performance for typical data, unless there is a lot of indexed data
     /// coincident with the shape's edge.
+    /// 
+    /// @lucene.experimental
     /// </summary>
-    /// <lucene.experimental></lucene.experimental>
     public class RecursivePrefixTreeStrategy : PrefixTreeStrategy
     {
         private int prefixGridScanLevel;
-        
-        /** True if only indexed points shall be supported.  See
-        *  {@link IntersectsPrefixTreeFilter#hasIndexedLeaves}. */
+
+        /// <summary>
+        /// True if only indexed points shall be supported.  See <see cref="IntersectsPrefixTreeFilter.hasIndexedLeaves"/>.
+        /// </summary>
         protected bool pointsOnly = false;
 
-        /** See {@link ContainsPrefixTreeFilter#multiOverlappingIndexedShapes}. */
+        /// <summary>
+        /// See <see cref="ContainsPrefixTreeFilter.multiOverlappingIndexedShapes"/>.
+        /// </summary>
         protected bool multiOverlappingIndexedShapes = true;
 
         public RecursivePrefixTreeStrategy(SpatialPrefixTree grid, string fieldName)
@@ -53,8 +53,8 @@ namespace Lucene.Net.Spatial.Prefix
 
         /// <summary>
         /// Sets the grid level [1-maxLevels] at which indexed terms are scanned brute-force
-        /// instead of by grid decomposition.By default this is maxLevels - 4.  The
-        /// final level, maxLevels, is always scanned.
+        /// instead of by grid decomposition. By default this is maxLevels - 4.  The
+        /// final level, maxLevels, is always scanned. Value can be 1 to maxLevels.
         /// </summary>
         public virtual int PrefixGridScanLevel
         {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
index 77afa1e..70ef71a 100644
--- a/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
@@ -26,22 +26,17 @@ namespace Lucene.Net.Spatial.Prefix
      */
 
     /// <summary>
-    /// A basic implementation of
-    /// <see cref="PrefixTreeStrategy">PrefixTreeStrategy</see>
-    /// using a large
-    /// <see cref="Lucene.Net.Queries.TermsFilter">Lucene.Net.Queries.TermsFilter
-    /// 	</see>
-    /// of all the cells from
-    /// <see cref="Lucene.Net.Spatial.Prefix.Tree.SpatialPrefixTree.GetCells(Shape, int, bool, bool)
-    /// 	"SpatialPrefixTree.GetCells(Shape, int, bool, bool)
-    /// 	</see>
-    /// . It only supports the search of indexed Point shapes.
-    /// <p/>
-    /// The precision of query shapes (distErrPct) is an important factor in using
+    /// A basic implementation of <see cref="PrefixTreeStrategy"/> using a large
+    /// <see cref="Lucene.Net.Queries.TermsFilter"/> of all the cells from
+    /// <see cref="Lucene.Net.Spatial.Prefix.Tree.SpatialPrefixTree.GetCells(IShape, int, bool, bool)"/>. 
+    /// It only supports the search of indexed Point shapes.
+    /// <para/>
+    /// The precision of query shapes (DistErrPct) is an important factor in using
     /// this Strategy. If the precision is too precise then it will result in many
     /// terms which will amount to a slower query.
+    /// 
+    /// @lucene.experimental
     /// </summary>
-    /// <lucene.experimental></lucene.experimental>
     public class TermQueryPrefixTreeStrategy : PrefixTreeStrategy
     {
         public TermQueryPrefixTreeStrategy(SpatialPrefixTree grid, string fieldName)

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs b/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
index 1e5e488..91db4d5 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
@@ -24,12 +24,12 @@ namespace Lucene.Net.Spatial.Prefix.Tree
      * limitations under the License.
      */
 
-    /// <summary>Represents a grid cell.</summary>
-    /// <remarks>
+    /// <summary>
     /// Represents a grid cell. These are not necessarily thread-safe, although new
     /// Cell("") (world cell) must be.
-    /// </remarks>
-    /// <lucene.experimental></lucene.experimental>
+    /// 
+    /// @lucene.experimental
+    /// </summary>
     public abstract class Cell : IComparable<Cell>
     {
         /// <summary>
@@ -44,10 +44,10 @@ namespace Lucene.Net.Spatial.Prefix.Tree
 
         public const byte LEAF_BYTE = (byte)('+');//NOTE: must sort before letters & numbers
 
-        /*
-        Holds a byte[] and/or String representation of the cell. Both are lazy constructed from the other.
-        Neither contains the trailing leaf byte.
-        */
+        /// <summary>
+        /// Holds a byte[] and/or String representation of the cell. Both are lazy constructed from the other.
+        /// Neither contains the trailing leaf byte.
+        /// </summary>
         private byte[] bytes;
         private int b_off;
         private int b_len;
@@ -55,10 +55,10 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         private string token;//this is the only part of equality
 
         /// <summary>
-        /// When set via getSubCells(filter), it is the relationship between this cell
+        /// When set via <see cref="GetSubCells(IShape)">GetSubCells(filter)</see>, it is the relationship between this cell
         /// and the given shape filter.
         /// </summary>
-        protected internal SpatialRelation shapeRel = SpatialRelation.NULL_VALUE;//set in getSubCells(filter), and via setLeaf().
+        protected internal SpatialRelation shapeRel = SpatialRelation.NULL_VALUE;//set in GetSubCells(filter), and via SetLeaf().
 
         /// <summary>Always false for points.</summary>
         /// <remarks>
@@ -142,7 +142,6 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         }
 
         /// <summary>Note: not supported at level 0.</summary>
-        /// <remarks>Note: not supported at level 0.</remarks>
         public virtual void SetLeaf()
         {
             Debug.Assert(Level != 0);
@@ -192,24 +191,14 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         //TODO add getParent() and update some algorithms to use this?
         //public Cell getParent();
         /// <summary>
-        /// Like
-        /// <see cref="GetSubCells()">GetSubCells()</see>
-        /// but with the results filtered by a shape. If
-        /// that shape is a
-        /// <see cref="Point">Spatial4n.Core.Shapes.Point</see>
-        /// then it must call
-        /// <see cref="GetSubCell(Point)">GetSubCell(Spatial4n.Core.Shapes.Point)
-        /// 	</see>
-        /// . The returned cells
-        /// should have
-        /// <see cref="ShapeRel">ShapeRel</see>
-        /// set to their relation with
-        /// <code>shapeFilter</code>
-        /// . In addition,
-        /// <see cref="IsLeaf()">IsLeaf()</see>
-        /// must be true when that relation is WITHIN.
-        /// <p/>
-        /// Precondition: Never called when getLevel() == maxLevel.
+        /// Like <see cref="GetSubCells()">GetSubCells()</see> but with the results filtered by a shape. If
+        /// that shape is a <see cref="IPoint"/> then it must call 
+        /// <see cref="GetSubCell(IPoint)"/>. The returned cells
+        /// should have <see cref="ShapeRel">ShapeRel</see> set to their relation with
+        /// <paramref name="shapeFilter"/>. In addition, <see cref="IsLeaf"/>
+        /// must be true when that relation is <see cref="SpatialRelation.WITHIN"/>.
+        /// <para/>
+        /// Precondition: Never called when Level == maxLevel.
         /// </summary>
         /// <param name="shapeFilter">an optional filter for the returned cells.</param>
         /// <returns>A set of cells (no dups), sorted. Not Modifiable.</returns>
@@ -258,9 +247,9 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// <remarks>
         /// Performant implementations are expected to implement this efficiently by
         /// considering the current cell's boundary. Precondition: Never called when
-        /// getLevel() == maxLevel.
+        /// Level == maxLevel.
         /// <p/>
-        /// Precondition: this.getShape().relate(p) != DISJOINT.
+        /// Precondition: this.Shape.Relate(p) != SpatialRelation.DISJOINT.
         /// </remarks>
         public abstract Cell GetSubCell(IPoint p);
 
@@ -268,14 +257,13 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// <summary>Gets the cells at the next grid cell level that cover this cell.</summary>
         /// <remarks>
         /// Gets the cells at the next grid cell level that cover this cell.
-        /// Precondition: Never called when getLevel() == maxLevel.
+        /// Precondition: Never called when Level == maxLevel.
         /// </remarks>
         /// <returns>A set of cells (no dups), sorted, modifiable, not empty, not null.</returns>
         protected internal abstract ICollection<Cell> GetSubCells();
 
         /// <summary>
-        /// <see cref="GetSubCells()">GetSubCells()</see>
-        /// .size() -- usually a constant. Should be &gt;=2
+        /// <see cref="GetSubCells()"/>.Count -- usually a constant. Should be &gt;=2
         /// </summary>
         public abstract int SubCellsSize { get; }
 

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
index 5dce614..d67a98e 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
@@ -24,23 +24,18 @@ namespace Lucene.Net.Spatial.Prefix.Tree
      */
 
     /// <summary>
-    /// A
-    /// <see cref="SpatialPrefixTree">SpatialPrefixTree</see>
-    /// based on
+    /// A <see cref="SpatialPrefixTree">SpatialPrefixTree</see> based on
     /// <a href="http://en.wikipedia.org/wiki/Geohash">Geohashes</a>.
-    /// Uses
-    /// <see cref="Spatial4n.Core.IO.GeohashUtils">Spatial4n.Core.IO.GeohashUtils
-    /// 	</see>
-    /// to do all the geohash work.
+    /// Uses <see cref="GeohashUtils"/> to do all the geohash work.
+    /// 
+    /// @lucene.experimental
     /// </summary>
-    /// <lucene.experimental></lucene.experimental>
     public class GeohashPrefixTree : SpatialPrefixTree
     {
         #region Nested type: Factory
 
         /// <summary>
-        /// Factory for creating
-        /// <see cref="GeohashPrefixTree">GeohashPrefixTree</see>
+        /// Factory for creating <see cref="GeohashPrefixTree"/>
         /// instances with useful defaults
         /// </summary>
         public class Factory : SpatialPrefixTreeFactory

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
index 1e90dc2..27787ef 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
@@ -26,21 +26,18 @@ namespace Lucene.Net.Spatial.Prefix.Tree
      */
 
     /// <summary>
-    /// A
-    /// <see cref="SpatialPrefixTree">SpatialPrefixTree</see>
-    /// which uses a
+    /// A <see cref="SpatialPrefixTree"/> which uses a
     /// <a href="http://en.wikipedia.org/wiki/Quadtree">quad tree</a> in which an
     /// indexed term will be generated for each cell, 'A', 'B', 'C', 'D'.
+    /// 
+    /// @lucene.experimental
     /// </summary>
-    /// <lucene.experimental></lucene.experimental>
     public class QuadPrefixTree : SpatialPrefixTree
     {
         #region Nested type: Factory
 
         /// <summary>
-        /// Factory for creating
-        /// <see cref="QuadPrefixTree">QuadPrefixTree</see>
-        /// instances with useful defaults
+        /// Factory for creating <see cref="QuadPrefixTree"/> instances with useful defaults
         /// </summary>
         public class Factory : SpatialPrefixTreeFactory
         {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
index 3fdac52..33925da 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
@@ -36,8 +36,9 @@ namespace Lucene.Net.Spatial.Prefix.Tree
     /// <p/>
     /// Implementations of this class should be thread-safe and immutable once
     /// initialized.
+    /// 
+    /// @lucene.experimental
     /// </remarks>
-    /// <lucene.experimental></lucene.experimental>
     public abstract class SpatialPrefixTree
     {
         protected internal readonly int maxLevels;
@@ -73,8 +74,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// <remarks>
         /// Returns the level of the largest grid in which its longest side is less
         /// than or equal to the provided distance (in degrees). Consequently
-        /// <code>dist</code>
-        /// acts as an error epsilon declaring the amount of detail needed in the
+        /// <paramref name="dist"/> acts as an error epsilon declaring the amount of detail needed in the
         /// grid, such that you can get a grid with just the right amount of
         /// precision.
         /// </remarks>
@@ -115,12 +115,11 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// <summary>Returns the level 0 cell which encompasses all spatial data.</summary>
         /// <remarks>
         /// Returns the level 0 cell which encompasses all spatial data. Equivalent to
-        /// <see cref="GetCell(string)">GetCell(string)</see>
-        /// with "".
+        /// <see cref="GetCell(string)">GetCell(string)</see> with <see cref="string.Empty"/>.
         /// This cell is threadsafe, just like a spatial prefix grid is, although cells aren't
         /// generally threadsafe.
-        /// TODO rename to getTopCell or is this fine?
         /// </remarks>
+        /// TODO rename to GetTopCell or is this fine?
         public virtual Cell WorldCell
         {
             get
@@ -136,8 +135,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// <summary>The cell for the specified token.</summary>
         /// <remarks>
         /// The cell for the specified token. The empty string should be equal to
-        /// <see cref="WorldCell">WorldCell</see>
-        /// .
+        /// <see cref="WorldCell">WorldCell</see>.
         /// Precondition: Never called when token length &gt; maxLevel.
         /// </remarks>
         public abstract Cell GetCell(string token);
@@ -155,11 +153,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         }
 
         /// <summary>
-        /// Returns the cell containing point
-        /// <code>p</code>
-        /// at the specified
-        /// <code>level</code>
-        /// .
+        /// Returns the cell containing point <paramref name="p"/> at the specified <paramref name="level"/>.
         /// </summary>
         protected internal virtual Cell GetCell(IPoint p, int level)
         {
@@ -174,11 +168,9 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// Gets the intersecting cells for the specified shape, without exceeding
         /// detail level. If a cell is within the query shape then it's marked as a
         /// leaf and none of its children are added.
-        /// <p/>
-        /// This implementation checks if shape is a Point and if so returns
-        /// <see cref="GetCells(Point, int, bool)">GetCells(Point, int, bool)
-        /// 	</see>
-        /// .
+        /// <para/>
+        /// This implementation checks if shape is a <see cref="IPoint"/> and if so returns
+        /// <see cref="GetCells(Point, int, bool)"/>.
         /// </remarks>
         /// <param name="shape">the shape; non-null</param>
         /// <param name="detailLevel">the maximum detail level to get cells for</param>
@@ -266,14 +258,10 @@ namespace Lucene.Net.Spatial.Prefix.Tree
 
         /// <summary>
         /// A Point-optimized implementation of
-        /// <see cref="GetCells(Shape, int, bool, bool)">GetCells(Shape, int, bool, bool)
-        /// 	</see>
-        /// . That
+        /// <see cref="GetCells(Shape, int, bool, bool)"/>. That
         /// method in facts calls this for points.
-        /// <p/>
-        /// This implementation depends on
-        /// <see cref="GetCell(string)">GetCell(string)</see>
-        /// being fast, as its
+        /// <para/>
+        /// This implementation depends on <see cref="GetCell(string)"/> being fast, as its
         /// called repeatedly when incPlarents is true.
         /// </summary>
         public virtual IList<Cell> GetCells(IPoint p, int detailLevel, bool inclParents)
@@ -298,9 +286,8 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             return cells;
         }
 
-        /// <summary>Will add the trailing leaf byte for leaves.</summary>
-        /// <remarks>Will add the trailing leaf byte for leaves. This isn't particularly efficient.
-        /// 	</remarks>
+        /// <summary>Will add the trailing leaf byte for leaves. This isn't particularly efficient.</summary>
+        [Obsolete("TODO remove; not used and not interesting, don't need collection in & out")]
         public static IList<string> CellsToTokenStrings(ICollection<Cell> cells)
         {
             IList<string> tokens = new List<string>((cells.Count));

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
index 34984b4..1103a6d 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
@@ -24,12 +24,11 @@ namespace Lucene.Net.Spatial.Prefix.Tree
      */
 
     /// <summary>
-    /// Abstract Factory for creating
-    /// <see cref="SpatialPrefixTree">SpatialPrefixTree</see>
-    /// instances with useful
-    /// defaults and passed on configurations defined in a Map.
+    /// Abstract Factory for creating <see cref="SpatialPrefixTree"/> instances with useful
+    /// defaults and passed on configurations defined in a <see cref="IDictionary{TKey, TValue}"/>.
+    /// 
+    /// @lucene.experimental
     /// </summary>
-    /// <lucene.experimental></lucene.experimental>
     public abstract class SpatialPrefixTreeFactory
     {
         private const double DEFAULT_GEO_MAX_DETAIL_KM = 0.001;//1m
@@ -41,8 +40,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         protected internal SpatialContext ctx;
         protected internal int? maxLevels;
 
-        /// <summary>The factory  is looked up via "prefixTree" in args, expecting "geohash" or "quad".
-        /// 	</summary>
+        /// <summary>The factory  is looked up via "prefixTree" in args, expecting "geohash" or "quad".</summary>
         /// <remarks>
         /// The factory  is looked up via "prefixTree" in args, expecting "geohash" or "quad".
         /// If its neither of these, then "geohash" is chosen for a geo context, otherwise "quad" is chosen.
@@ -113,8 +111,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         }
 
         /// <summary>
-        /// Calls
-        /// <see cref="SpatialPrefixTree.GetLevelForDistance(double)">SpatialPrefixTree.GetLevelForDistance(double)</see>.
+        /// Calls <see cref="SpatialPrefixTree.GetLevelForDistance(double)">SpatialPrefixTree.GetLevelForDistance(double)</see>.
         /// </summary>
         protected internal abstract int GetLevelForDistance(double degrees);
 

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
index fc9f5b3..0cf7b6e 100644
--- a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
@@ -30,22 +30,19 @@ namespace Lucene.Net.Spatial.Prefix
 
     /// <summary>
     /// Finds docs where its indexed shape is
-    /// <see cref="SpatialOperation.IsWithin">WITHIN</see>
+    /// <see cref="Queries.SpatialOperation.IsWithin">WITHIN</see>
     /// the query shape.  It works by looking at cells outside of the query
     /// shape to ensure documents there are excluded. By default, it will
     /// examine all cells, and it's fairly slow.  If you know that the indexed shapes
     /// are never comprised of multiple disjoint parts (which also means it is not multi-valued),
-    /// then you can pass
-    /// <code>SpatialPrefixTree.getDistanceForLevel(maxLevels)</code>
-    /// as
-    /// the
-    /// <code>queryBuffer</code>
-    /// constructor parameter to minimally look this distance
+    /// then you can pass <c>SpatialPrefixTree.GetDistanceForLevel(maxLevels)</c> as
+    /// the <c>queryBuffer</c> constructor parameter to minimally look this distance
     /// beyond the query shape's edge.  Even if the indexed shapes are sometimes
     /// comprised of multiple disjoint parts, you might want to use this option with
     /// a large buffer as a faster approximation with minimal false-positives.
+    /// 
+    /// @lucene.experimental
     /// </summary>
-    /// <lucene.experimental></lucene.experimental>
     public class WithinPrefixTreeFilter : AbstractVisitingPrefixTreeFilter
     {
         /// TODO LUCENE-4869: implement faster algorithm based on filtering out false-positives of a
@@ -55,12 +52,8 @@ namespace Lucene.Net.Spatial.Prefix
         private readonly IShape bufferedQueryShape;//if null then the whole world
 
         /// <summary>
-        /// See
-        /// <see cref="AbstractVisitingPrefixTreeFilter">AbstractVisitingPrefixTreeFilter.AbstractVisitingPrefixTreeFilter(Shape, string, Lucene.Net.Spatial.Prefix.Tree.SpatialPrefixTree, int, int)
-        /// 	</see>
-        /// .
-        /// <code>queryBuffer</code>
-        /// is the (minimum) distance beyond the query shape edge
+        /// See <see cref="AbstractVisitingPrefixTreeFilter.AbstractVisitingPrefixTreeFilter(IShape, string, SpatialPrefixTree, int, int)"/>.
+        /// <c>queryBuffer</c> is the (minimum) distance beyond the query shape edge
         /// where non-matching documents are looked for so they can be excluded. If
         /// -1 is used then the whole world is examined (a good default for correctness).
         /// </summary>
@@ -79,7 +72,7 @@ namespace Lucene.Net.Spatial.Prefix
         }
 
         /// <summary>
-        /// Returns a new shape that is larger than shape by at distErr
+        /// Returns a new shape that is larger than shape by at distErr.
         /// </summary>
         protected virtual IShape BufferShape(IShape shape, double distErr)
         {
@@ -224,10 +217,6 @@ namespace Lucene.Net.Spatial.Prefix
             /// Returns true if the provided cell, and all its sub-cells down to
             /// detailLevel all intersect the queryShape.
             /// </summary>
-            /// <remarks>
-            /// Returns true if the provided cell, and all its sub-cells down to
-            /// detailLevel all intersect the queryShape.
-            /// </remarks>
             private bool AllCellsIntersectQuery(Cell cell, SpatialRelation relate/*cell to query*/)
             {
                 if (relate == SpatialRelation.NULL_VALUE)

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialArgs.cs b/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
index 7c49f5d..a056341 100644
--- a/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
+++ b/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
@@ -21,6 +21,12 @@ namespace Lucene.Net.Spatial.Queries
      * limitations under the License.
      */
 
+    /// <summary>
+    /// Principally holds the query <see cref="IShape"/> and the <see cref="SpatialOperation"/>.
+    /// It's used as an argument to some methods on <see cref="SpatialStrategy"/>.
+    /// 
+    /// @lucene.experimental
+    /// </summary>
     public class SpatialArgs
     {
         public static readonly double DEFAULT_DISTERRPCT = 0.025d;
@@ -39,7 +45,7 @@ namespace Lucene.Net.Spatial.Queries
         }
 
         /// <summary>
-        /// Computes the distance given a shape and the {@code distErrPct}.  The
+        /// Computes the distance given a shape and the <paramref name="distErrPct"/>.  The
         /// algorithm is the fraction of the distance from the center of the query
         /// shape to its furthest bounding box corner.
         /// </summary>
@@ -75,7 +81,7 @@ namespace Lucene.Net.Spatial.Queries
         /// </summary>
         /// <param name="ctx"></param>
         /// <param name="defaultDistErrPct">0 to 0.5</param>
-        /// <returns>>= 0</returns>
+        /// <returns>&gt;= 0</returns>
         public virtual double ResolveDistErr(SpatialContext ctx, double defaultDistErrPct)
         {
             if (DistErr != null)
@@ -89,7 +95,7 @@ namespace Lucene.Net.Spatial.Queries
         /// </summary>
         public virtual void Validate()
         {
-            if (Operation.TargetNeedsArea && !Shape.HasArea)
+            if (Operation.IsTargetNeedsArea && !Shape.HasArea)
             {
                 throw new ArgumentException(Operation + " only supports geometry with area");
             }
@@ -124,8 +130,6 @@ namespace Lucene.Net.Spatial.Queries
         /// <summary>
         /// A measure of acceptable error of the shape as a fraction. This effectively
         /// inflates the size of the shape but should not shrink it.
-        /// <p/>
-        /// The default is {@link #DEFAULT_DIST_PRECISION}
         /// </summary>
         /// <returns>0 to 0.5</returns>
         public virtual double? DistErrPct
@@ -144,7 +148,7 @@ namespace Lucene.Net.Spatial.Queries
         /// The acceptable error of the shape.  This effectively inflates the
         /// size of the shape but should not shrink it.
         /// </summary>
-        /// <returns>>= 0</returns>
+        /// <returns>&gt;= 0</returns>
         public virtual double? DistErr
         {
             get { return distErr; }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs b/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
index db0f4ca..6a1a3d8 100644
--- a/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
+++ b/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
@@ -27,10 +27,10 @@ namespace Lucene.Net.Spatial.Queries
      */
 
     /// <summary>
-    /// Parses a string that usually looks like "OPERATION(SHAPE)" into a {@link SpatialArgs}
-    /// object. The set of operations supported are defined in {@link SpatialOperation}, such
-    /// as "Intersects" being a common one. The shape portion is defined by WKT {@link com.spatial4j.core.io.WktShapeParser},
-    /// but it can be overridden/customized via {@link #parseShape(String, com.spatial4j.core.context.SpatialContext)}.
+    /// Parses a string that usually looks like "OPERATION(SHAPE)" into a <see cref="SpatialArgs"/>
+    /// object. The set of operations supported are defined in <see cref="SpatialOperation"/>, such
+    /// as "Intersects" being a common one. The shape portion is defined by WKT <see cref="Spatial4n.Core.Io.WktShapeParser"/>,
+    /// but it can be overridden/customized via <see cref="ParseShape(string, SpatialContext)"/>.
     /// There are some optional name-value pair parameters that follow the closing parenthesis.  Example:
     /// <code>
     ///   Intersects(ENVELOPE(-10,-8,22,20)) distErrPct=0.025
@@ -65,7 +65,7 @@ namespace Lucene.Net.Spatial.Queries
         }
 
         /// <summary>
-        /// Parses a string such as "Intersects(-10,20,-8,22) distErrPct=0.025".
+        /// Parses a string such as "Intersects(ENVELOPE(-10,-8,22,20)) distErrPct=0.025".
         /// </summary>
         /// <param name="v">The string to parse. Mandatory.</param>
         /// <param name="ctx">The spatial context. Mandatory.</param>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialOperation.cs b/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
index a05703e..d2e204a 100644
--- a/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
+++ b/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
@@ -201,7 +201,7 @@ namespace Lucene.Net.Spatial.Queries
         }
 
         /// <summary>
-        /// Returns whether the relationship between indexedShape and queryShape is
+        /// Returns whether the relationship between <paramref name="indexedShape"/> and <paramref name="queryShape"/> is
         /// satisfied by this operation.
         /// </summary>
         public abstract bool Evaluate(IShape indexedShape, IShape queryShape);
@@ -218,7 +218,7 @@ namespace Lucene.Net.Spatial.Queries
             get { return sourceNeedsArea; }
         }
 
-        public virtual bool TargetNeedsArea
+        public virtual bool IsTargetNeedsArea
         {
             get { return targetNeedsArea; }
         }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs b/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs
index 8d2a107..2b658f0 100644
--- a/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs
+++ b/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs
@@ -19,6 +19,11 @@ namespace Lucene.Net.Spatial.Queries
      * limitations under the License.
      */
 
+    /// <summary>
+    /// Exception thrown when the <see cref="SpatialStrategy"/> cannot implement the requested operation.
+    /// 
+    /// @lucene.experimental
+    /// </summary>
     [Serializable]
     public class UnsupportedSpatialOperation : NotSupportedException
     {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
index d85ce1e..e401c34 100644
--- a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
+++ b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
@@ -33,12 +33,12 @@ namespace Lucene.Net.Spatial.Serialized
      */
 
     /// <summary>
-    /// A SpatialStrategy based on serializing a Shape stored into BinaryDocValues.
+    /// A <see cref="SpatialStrategy"/> based on serializing a Shape stored into BinaryDocValues.
     /// This is not at all fast; it's designed to be used in conjuction with another index based
-    /// SpatialStrategy that is approximated(like { @link org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy})
+    /// SpatialStrategy that is approximated(like <see cref="Prefix.RecursivePrefixTreeStrategy"/>)
     /// to add precision or eventually make more specific / advanced calculations on the per-document
     /// geometry.
-    /// The serialization uses Spatial4j's {@link com.spatial4j.core.io.BinaryCodec}.
+    /// The serialization uses Spatial4j's <see cref="BinaryCodec"/>.
     ///
     /// @lucene.experimental
     /// </summary>
@@ -108,7 +108,7 @@ namespace Lucene.Net.Spatial.Serialized
         }
 
         /// <summary>
-        /// Returns a Filter that should be used with <see cref="FilteredQuery.QUERY_FIRST_FILTER_STRATEGY"/>.
+        /// Returns a <see cref="Filter"/> that should be used with <see cref="FilteredQuery.QUERY_FIRST_FILTER_STRATEGY"/>.
         /// Use in another manner is likely to result in an <see cref="NotSupportedException"/>
         /// to prevent misuse because the filter can't efficiently work via iteration.
         /// </summary>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/SpatialStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/SpatialStrategy.cs b/src/Lucene.Net.Spatial/SpatialStrategy.cs
index a9bbde1..14b67e2 100644
--- a/src/Lucene.Net.Spatial/SpatialStrategy.cs
+++ b/src/Lucene.Net.Spatial/SpatialStrategy.cs
@@ -27,7 +27,7 @@ namespace Lucene.Net.Spatial
      */
 
     /// <summary>
-    /// The SpatialStrategy encapsulates an approach to indexing and searching based on shapes.
+    /// The <see cref="SpatialStrategy"/> encapsulates an approach to indexing and searching based on shapes.
     /// <para/>
     /// Different implementations will support different features. A strategy should
     /// document these common elements:
@@ -76,33 +76,34 @@ namespace Lucene.Net.Spatial
         /// The name of the field or the prefix of them if there are multiple
         /// fields needed internally.
         /// </summary>
-        /// <returns></returns>
+        /// <returns>Not null.</returns>
         public virtual string FieldName
         {
             get { return fieldName; }
         }
 
         /// <summary>
-        /// Returns the IndexableField(s) from the <c>shape</c> that are to be
-        /// added to the {@link org.apache.lucene.document.Document}.  These fields
+        /// Returns the IndexableField(s) from the <paramref name="shape"/> that are to be
+        /// added to the <see cref="Document"/>.  These fields
         /// are expected to be marked as indexed and not stored.
         /// <p/>
         /// Note: If you want to <i>store</i> the shape as a string for retrieval in search
         /// results, you could add it like this:
-        /// <pre>document.add(new StoredField(fieldName,ctx.toString(shape)));</pre>
+        /// <code>
+        ///     document.Add(new StoredField(fieldName, ctx.ToString(shape)));
+        /// </code>
         /// The particular string representation used doesn't matter to the Strategy since it
         /// doesn't use it.
         /// </summary>
         /// <param name="shape"></param>
         /// <returns>Not null nor will it have null elements.</returns>
+        /// <exception cref="NotSupportedException">if given a shape incompatible with the strategy</exception>
         public abstract Field[] CreateIndexableFields(IShape shape);
 
         /// <summary>
-        /// See {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point, double)} called with
+        /// See <see cref="MakeDistanceValueSource(IPoint, double)"/> called with
         /// a multiplier of 1.0 (i.e. units of degrees).
         /// </summary>
-        /// <param name="queryPoint"></param>
-        /// <returns></returns>
         public virtual ValueSource MakeDistanceValueSource(IPoint queryPoint)
         {
             return MakeDistanceValueSource(queryPoint, 1.0);
@@ -110,41 +111,42 @@ namespace Lucene.Net.Spatial
 
         /// <summary>
         /// 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.
+        /// indexed shape and <paramref name="queryPoint"/>.  If there are multiple indexed shapes
+        /// then the closest one is chosen. The result is multiplied by <paramref name="multiplier"/>, which
+        /// conveniently is used to get the desired units.
         /// </summary>
         public abstract ValueSource MakeDistanceValueSource(IPoint queryPoint, double multiplier);
 
         /// <summary>
-        /// Make a (ConstantScore) Query based principally on {@link org.apache.lucene.spatial.query.SpatialOperation}
-        /// and {@link Shape} from the supplied {@code args}.
+        /// Make a Query based principally on <see cref="SpatialOperation"/>
+        /// and <see cref="IShape"/> from the supplied <paramref name="args"/>.
         /// The default implementation is
-        /// <pre>return new ConstantScoreQuery(makeFilter(args));</pre>
+        /// <code>return new ConstantScoreQuery(MakeFilter(args));</code>
         /// </summary>
-        /// <param name="args"></param>
-        /// <returns></returns>
+        /// <exception cref="NotSupportedException">If the strategy does not support the shape in <paramref name="args"/>.</exception>
+        /// <exception cref="UnsupportedSpatialOperation">If the strategy does not support the <see cref="SpatialOperation"/> in <paramref name="args"/>.</exception>
         public virtual ConstantScoreQuery MakeQuery(SpatialArgs args)
         {
             return new ConstantScoreQuery(MakeFilter(args));
         }
 
         /// <summary>
-        /// Make a Filter based principally on {@link org.apache.lucene.spatial.query.SpatialOperation}
-        /// and {@link Shape} from the supplied {@code args}.
-        /// <p />
+        /// Make a Filter based principally on <see cref="SpatialOperation"/>
+        /// and <see cref="IShape"/> from the supplied <paramref name="args"/>.
+        /// <para />
         /// If a subclasses implements
-        /// {@link #makeQuery(org.apache.lucene.spatial.query.SpatialArgs)}
+        /// <see cref="MakeQuery(SpatialArgs)"/>
         /// then this method could be simply:
-        /// <pre>return new QueryWrapperFilter(makeQuery(args).getQuery());</pre>
+        /// <code>return new QueryWrapperFilter(MakeQuery(args).Query);</code>
         /// </summary>
-        /// <param name="args"></param>
-        /// <returns></returns>
+        /// <exception cref="NotSupportedException">If the strategy does not support the shape in <paramref name="args"/>.</exception>
+        /// <exception cref="UnsupportedSpatialOperation">If the strategy does not support the <see cref="SpatialOperation"/> in <paramref name="args"/>.</exception>
         public abstract Filter MakeFilter(SpatialArgs args);
 
         /// <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 <c>c/(d + c)</c> where 'd' is the distance and 'c' is
+        /// on the distance from <see cref="MakeDistanceValueSource(IPoint)"/>.
+        /// The formula is <c>c / (d + c)</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
         /// low as ~0.1 at its furthest edges.

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs b/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs
index dc0a5d3..cf48502 100644
--- a/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs
+++ b/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs
@@ -25,7 +25,7 @@ namespace Lucene.Net.Spatial.Util
     /// associated with a given docId
     /// <para/>
     /// WARNING: This class holds the data in an extremely inefficient manner as all Points are in memory as objects and they
-    /// are stored in many ArrayLists (one per document).  So it works but doesn't scale.  It will be replaced in the future.
+    /// are stored in many Lists (one per document).  So it works but doesn't scale.  It will be replaced in the future.
     /// <para/>
     /// @lucene.internal
     /// </summary>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Util/ShapePredicateValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ShapePredicateValueSource.cs b/src/Lucene.Net.Spatial/Util/ShapePredicateValueSource.cs
index 8914343..9065979 100644
--- a/src/Lucene.Net.Spatial/Util/ShapePredicateValueSource.cs
+++ b/src/Lucene.Net.Spatial/Util/ShapePredicateValueSource.cs
@@ -26,7 +26,7 @@ namespace Lucene.Net.Spatial.Util
      */
 
     /// <summary>
-    /// A boolean ValueSource that compares a shape from a provided ValueSource with a given Shape and sees
+    /// A boolean <see cref="ValueSource"/> that compares a shape from a provided <see cref="ValueSource"/> with a given <see cref="IShape">Shape</see> and sees
     /// if it matches a given <see cref="SpatialOperation"/> (the predicate).
     /// 
     /// @lucene.experimental
@@ -41,7 +41,7 @@ namespace Lucene.Net.Spatial.Util
         /// 
         /// </summary>
         /// <param name="shapeValuesource">
-        /// Must yield <see cref="IShape"/> instances from it's objectVal(doc). If null
+        /// Must yield <see cref="IShape"/> instances from it's ObjectVal(doc). If null
         /// then the result is false. This is the left-hand (indexed) side.
         /// </param>
         /// <param name="op">the predicate</param>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs b/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
index 1abfa09..fcce878 100644
--- a/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
+++ b/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
@@ -24,7 +24,7 @@ namespace Lucene.Net.Spatial.Util
      */
 
     /// <summary>
-    /// Filter that matches all documents where a valuesource is
+    /// <see cref="Filter"/> that matches all documents where a <see cref="ValueSource"/> is
     /// in between a range of <c>min</c> and <c>max</c> inclusive.
     /// @lucene.internal
     /// </summary>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs b/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
index 6f1dfb6..4d1dfc6 100644
--- a/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
+++ b/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
@@ -27,7 +27,7 @@ namespace Lucene.Net.Spatial.Vector
      */
 
     /// <summary>
-    /// An implementation of the Lucene ValueSource model that returns the distance
+    /// An implementation of the Lucene <see cref="ValueSource"/> model that returns the distance
     /// for a <see cref="PointVectorStrategy"/>.
     /// 
     /// @lucene.internal
@@ -49,7 +49,7 @@ namespace Lucene.Net.Spatial.Vector
         }
 
         /// <summary>
-        /// Returns the ValueSource description.
+        /// Returns the <see cref="ValueSource"/> description.
         /// </summary>
         public override string Description
         {
@@ -57,7 +57,7 @@ namespace Lucene.Net.Spatial.Vector
         }
 
         /// <summary>
-        /// Returns the FunctionValues used by the function query.
+        /// Returns the <see cref="FunctionValues"/> used by the function query.
         /// </summary>
         public override FunctionValues GetValues(IDictionary context, AtomicReaderContext readerContext)
         {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
index 3526291..cafe4e6 100644
--- a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
+++ b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
@@ -28,13 +28,28 @@ namespace Lucene.Net.Spatial.Vector
 
     /// <summary>
     /// Simple <see cref="SpatialStrategy"/> which represents Points in two numeric <see cref="DoubleField"/>s.
+    /// The Strategy's best feature is decent distance sort.
     /// 
-    /// Note, currently only Points can be indexed by this Strategy.  At query time, the bounding
-    /// box of the given Shape is used to create <see cref="NumericRangeQuery"/>s to efficiently
-    /// find Points within the Shape.
+    /// <h4>Characteristics:</h4>
+    /// <list type="bullet">
+    ///     <item>Only indexes points; just one per field value.</item>
+    ///     <item>Can query by a rectangle or circle.</item>
+    ///     <item><see cref="SpatialOperation.Intersects"/> and <see cref="SpatialOperation.IsWithin"/> is supported.</item>
+    ///     <item>Uses the FieldCache for <see cref="SpatialStrategy.MakeDistanceValueSource(IPoint)"/> and for
+    ///     searching with a Circle.</item>
+    /// </list>
     /// 
-    /// Due to the simple use of numeric fields, this Strategy provides support for sorting by
-    /// distance through <see cref="DistanceValueSource"/>
+    /// <h4>Implementation:</h4>
+    /// This is a simple Strategy.  Search works with <see cref="NumericRangeQuery"/>s on
+    /// an x & y pair of fields.  A Circle query does the same bbox query but adds a
+    /// ValueSource filter on <see cref="SpatialStrategy.MakeDistanceValueSource(IPoint)"/>.
+    /// <para/>
+    /// One performance shortcoming with this strategy is that a scenario involving
+    /// both a search using a Circle and sort will result in calculations for the
+    /// spatial distance being done twice -- once for the filter and second for the
+    /// sort.
+    /// 
+    /// @lucene.experimental
     /// </summary>
     public class PointVectorStrategy : SpatialStrategy
     {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/8219d871/src/Lucene.Net.Tests.Spatial/Prefix/NtsPolygonTest.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Prefix/NtsPolygonTest.cs b/src/Lucene.Net.Tests.Spatial/Prefix/NtsPolygonTest.cs
index 7f4338d..50e60e2 100644
--- a/src/Lucene.Net.Tests.Spatial/Prefix/NtsPolygonTest.cs
+++ b/src/Lucene.Net.Tests.Spatial/Prefix/NtsPolygonTest.cs
@@ -44,7 +44,7 @@ namespace Lucene.Net.Spatial.Prefix
             }
             catch (TypeLoadException e) //LUCENENET TODO: Does this match NoClassDefFoundError ??
             {
-                AssumeTrue("This test requires JTS jar: " + e, false);
+                AssumeTrue("This test requires Spatial4n.Core.NTS: " + e, false);
             }
 
             GeohashPrefixTree grid = new GeohashPrefixTree(ctx, 11);//< 1 meter == 11 maxLevels


[19/26] lucenenet git commit: Fixed bug in assert statement that was causing it to incorrectly fail

Posted by ni...@apache.org.
Fixed bug in assert statement that was causing it to incorrectly fail


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

Branch: refs/heads/master
Commit: a0b447fb97f1802dbd0ea080a6973933eb2e2fd5
Parents: df3f64d
Author: Shad Storhaug <sh...@shadstorhaug.com>
Authored: Thu Nov 17 22:27:03 2016 +0700
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Fri Nov 18 01:35:49 2016 +0700

----------------------------------------------------------------------
 src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/a0b447fb/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
index b36604e..466cab8 100644
--- a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
@@ -168,7 +168,7 @@ namespace Lucene.Net.Spatial.Prefix
             private SmallDocSet GetLeafDocs(Cell leafCell, Bits acceptContains)
             {
                 Debug.Assert(new BytesRef(leafCell.GetTokenBytes()).Equals(termBytes));
-                Debug.Assert(leafCell.Equals(lastLeaf));//don't call for same leaf again
+                Debug.Assert(!leafCell.Equals(lastLeaf));//don't call for same leaf again
                 lastLeaf = leafCell;
 
                 if (termsEnum == null)


[11/26] lucenenet git commit: Added missing Spatial classes, ported Spatial tests, and fixed several bugs

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Test-Files/Data/simple-bbox.txt
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Test-Files/Data/simple-bbox.txt b/src/Lucene.Net.Tests.Spatial/Test-Files/Data/simple-bbox.txt
new file mode 100644
index 0000000..174aa1b
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Test-Files/Data/simple-bbox.txt
@@ -0,0 +1,5 @@
+#id	name	shape	
+C5	CenterAt5	ENVELOPE(-5, 5, 5, -5)
+C10	CenterAt10	ENVELOPE(-10, 10, 10, -10)
+NW15	NorthWest	ENVELOPE(15, 20, 20, 15)
+

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Test-Files/Data/states-bbox.txt
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Test-Files/Data/states-bbox.txt b/src/Lucene.Net.Tests.Spatial/Test-Files/Data/states-bbox.txt
new file mode 100644
index 0000000..bfe041f
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Test-Files/Data/states-bbox.txt
@@ -0,0 +1,52 @@
+#id	name	shape	
+HI	Hawaii	ENVELOPE(-160.242406, -154.791096, 22.229120, 18.921786)
+WA	Washington	ENVELOPE(-124.732769, -116.919132, 48.999931, 45.543092)
+MT	Montana	ENVELOPE(-116.063531, -104.043072, 49.000026, 44.353639)
+ME	Maine	ENVELOPE(-71.087509, -66.969271, 47.453334, 43.091050)
+ND	North Dakota	ENVELOPE(-104.062991, -96.551931, 49.000026, 45.930822)
+SD	South Dakota	ENVELOPE(-104.061036, -96.439394, 45.943547, 42.488459)
+WY	Wyoming	ENVELOPE(-111.053428, -104.051705, 45.002793, 40.994289)
+WI	Wisconsin	ENVELOPE(-92.885397, -86.967712, 46.952479, 42.489152)
+ID	Idaho	ENVELOPE(-117.236921, -111.046771, 48.999950, 41.994599)
+VT	Vermont	ENVELOPE(-73.436000, -71.505372, 45.013351, 42.725852)
+MN	Minnesota	ENVELOPE(-97.229436, -89.530673, 49.371730, 43.498102)
+OR	Oregon	ENVELOPE(-124.559617, -116.470418, 46.236091, 41.987672)
+NH	New Hampshire	ENVELOPE(-72.553428, -70.734139, 45.301469, 42.698603)
+IA	Iowa	ENVELOPE(-96.640709, -90.142796, 43.501457, 40.371946)
+MA	Massachusetts	ENVELOPE(-73.498840, -69.917780, 42.886877, 41.238279)
+NE	Nebraska	ENVELOPE(-104.056219, -95.308697, 43.003062, 39.992595)
+NY	New York	ENVELOPE(-79.763235, -71.869986, 45.006138, 40.506003)
+PA	Pennsylvania	ENVELOPE(-80.526045, -74.700062, 42.267327, 39.719313)
+CT	Connecticut	ENVELOPE(-73.725237, -71.788249, 42.047428, 40.998392)
+RI	Rhode Island	ENVELOPE(-71.866678, -71.117132, 42.013713, 41.322769)
+NJ	New Jersey	ENVELOPE(-75.570234, -73.896148, 41.350573, 38.956682)
+IN	Indiana	ENVELOPE(-88.101490, -84.787446, 41.765540, 37.776224)
+NV	Nevada	ENVELOPE(-119.996324, -114.037392, 41.996637, 34.998914)
+UT	Utah	ENVELOPE(-114.047273, -109.043206, 42.002300, 36.991746)
+CA	California	ENVELOPE(-124.392638, -114.125230, 42.002191, 32.535781)
+OH	Ohio	ENVELOPE(-84.812070, -80.519996, 41.986872, 38.400511)
+IL	Illinois	ENVELOPE(-91.516284, -87.507909, 42.509363, 36.986822)
+DC	District of Columbia	ENVELOPE(-77.122328, -76.910904, 38.993541, 38.788234)
+DE	Delaware	ENVELOPE(-75.791094, -75.045623, 39.840119, 38.449602)
+WV	West Virginia	ENVELOPE(-82.647158, -77.727467, 40.637203, 37.204910)
+MD	Maryland	ENVELOPE(-79.489865, -75.045623, 39.725461, 37.970255)
+CO	Colorado	ENVELOPE(-109.055861, -102.037207, 41.003375, 36.988994)
+KY	Kentucky	ENVELOPE(-89.568231, -81.959575, 39.142063, 36.496570)
+KS	Kansas	ENVELOPE(-102.051535, -94.601224, 40.002987, 36.988875)
+VA	Virginia	ENVELOPE(-83.675177, -75.242219, 39.456998, 36.541623)
+MO	Missouri	ENVELOPE(-95.767479, -89.105034, 40.609784, 35.989656)
+AZ	Arizona	ENVELOPE(-114.821761, -109.045615, 37.003926, 31.335634)
+OK	Oklahoma	ENVELOPE(-102.997709, -94.428552, 37.001478, 33.621136)
+NC	North Carolina	ENVELOPE(-84.323773, -75.456580, 36.589767, 33.882164)
+TN	Tennessee	ENVELOPE(-90.305448, -81.652272, 36.679683, 34.988759)
+TX	Texas	ENVELOPE(-106.650062, -93.507389, 36.493912, 25.845557)
+NM	New Mexico	ENVELOPE(-109.051346, -102.997401, 36.999760, 31.343453)
+AL	Alabama	ENVELOPE(-88.472952, -84.894016, 35.016033, 30.233604)
+MS	Mississippi	ENVELOPE(-91.643682, -88.090468, 35.005041, 30.194935)
+GA	Georgia	ENVELOPE(-85.608960, -80.894753, 35.000366, 30.361291)
+SC	South Carolina	ENVELOPE(-83.350685, -78.579453, 35.208356, 32.068173)
+AR	Arkansas	ENVELOPE(-94.617257, -89.645479, 36.492811, 33.010151)
+LA	Louisiana	ENVELOPE(-94.041785, -89.021803, 33.023422, 28.939655)
+FL	Florida	ENVELOPE(-87.625711, -80.050911, 31.003157, 24.956376)
+MI	Michigan	ENVELOPE(-90.408200, -82.419836, 48.173795, 41.697494)
+AK	Alaska	ENVELOPE(-178.217598, -129.992235, 71.406235, 51.583032)


[21/26] lucenenet git commit: Spatial: Normalized license headers

Posted by ni...@apache.org.
Spatial: Normalized license headers


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

Branch: refs/heads/master
Commit: d21093a05ba030f97bcc2e0c217904eac5ba694c
Parents: 553851c
Author: Shad Storhaug <sh...@shadstorhaug.com>
Authored: Thu Nov 17 23:58:19 2016 +0700
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Fri Nov 18 01:38:20 2016 +0700

----------------------------------------------------------------------
 src/Lucene.Net.Spatial/DisjointSpatialFilter.cs | 33 +++++++++---------
 .../Prefix/AbstractPrefixTreeFilter.cs          | 33 +++++++++---------
 .../Prefix/AbstractVisitingPrefixTreeFilter.cs  | 33 +++++++++---------
 .../Prefix/ContainsPrefixTreeFilter.cs          | 33 +++++++++---------
 .../Prefix/IntersectsPrefixTreeFilter.cs        | 33 +++++++++---------
 .../Prefix/PointPrefixTreeFieldCacheProvider.cs | 33 +++++++++---------
 .../Prefix/PrefixTreeStrategy.cs                | 33 +++++++++---------
 .../Prefix/RecursivePrefixTreeStrategy.cs       | 34 +++++++++---------
 .../Prefix/TermQueryPrefixTreeStrategy.cs       | 33 +++++++++---------
 src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs      | 33 +++++++++---------
 .../Prefix/Tree/GeohashPrefixTree.cs            | 33 +++++++++---------
 .../Prefix/Tree/QuadPrefixTree.cs               | 33 +++++++++---------
 .../Prefix/Tree/SpatialPrefixTree.cs            | 33 +++++++++---------
 .../Prefix/Tree/SpatialPrefixTreeFactory.cs     | 33 +++++++++---------
 .../Prefix/WithinPrefixTreeFilter.cs            | 33 +++++++++---------
 src/Lucene.Net.Spatial/Query/SpatialArgs.cs     | 36 ++++++++++----------
 .../Query/SpatialArgsParser.cs                  | 36 ++++++++++----------
 .../Query/SpatialOperation.cs                   | 34 +++++++++---------
 .../Query/UnsupportedSpatialOperation.cs        | 36 ++++++++++----------
 .../Serialized/SerializedDVStrategy.cs          | 36 ++++++++++----------
 src/Lucene.Net.Spatial/SpatialStrategy.cs       | 36 ++++++++++----------
 .../Util/CachingDoubleValueSource.cs            | 36 ++++++++++----------
 src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs  | 36 ++++++++++----------
 .../Util/ShapeFieldCacheDistanceValueSource.cs  | 36 ++++++++++----------
 .../Util/ShapeFieldCacheProvider.cs             | 33 +++++++++---------
 .../Util/ValueSourceFilter.cs                   | 36 ++++++++++----------
 .../Vector/DistanceValueSource.cs               | 36 ++++++++++----------
 .../Vector/PointVectorStrategy.cs               | 36 ++++++++++----------
 28 files changed, 488 insertions(+), 471 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs b/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
index 9c0235e..509b5cf 100644
--- a/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
+++ b/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
@@ -1,19 +1,3 @@
-/* 
- * 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.Index;
 using Lucene.Net.Queries;
 using Lucene.Net.Search;
@@ -23,6 +7,23 @@ using System;
 
 namespace Lucene.Net.Spatial
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// A Spatial Filter implementing
     /// <see cref="SpatialOperation.IsDisjointTo">Org.Apache.Lucene.Spatial.Query.SpatialOperation.IsDisjointTo

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
index 08c5e4c..67f0c1b 100644
--- a/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
@@ -1,19 +1,3 @@
-/* 
- * 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.Index;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix.Tree;
@@ -23,6 +7,23 @@ using System.Diagnostics;
 
 namespace Lucene.Net.Spatial.Prefix
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// Base class for Lucene Filters on SpatialPrefixTree fields.
     /// @lucene.experimental

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
index af8920d..42c76d1 100644
--- a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
@@ -1,19 +1,3 @@
-/* 
- * 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.Index;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix.Tree;
@@ -26,6 +10,23 @@ using System.Diagnostics;
 
 namespace Lucene.Net.Spatial.Prefix
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// Traverses a <see cref="SpatialPrefixTree">SpatialPrefixTree</see> indexed field, using the template &
     /// visitor design patterns for subclasses to guide the traversal and collect

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
index 0c244cc..ea10507 100644
--- a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
@@ -1,19 +1,3 @@
-/* 
- * 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.Index;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix.Tree;
@@ -25,6 +9,23 @@ using System.Diagnostics;
 
 namespace Lucene.Net.Spatial.Prefix
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// Finds docs where its indexed shape
     /// <see cref="SpatialOperation.Contains">CONTAINS</see>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
index a150e37..f3d697b 100644
--- a/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
@@ -1,19 +1,3 @@
-/* 
- * 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.Index;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix.Tree;
@@ -22,6 +6,23 @@ using Spatial4n.Core.Shapes;
 
 namespace Lucene.Net.Spatial.Prefix
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// A Filter matching documents that have an
     /// <see cref="SpatialRelation.Intersects">SpatialRelation.Intersects

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs b/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
index e4cee6a..f89da09 100644
--- a/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
+++ b/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
@@ -1,19 +1,3 @@
-/* 
- * 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.Spatial.Prefix.Tree;
 using Lucene.Net.Spatial.Util;
 using Lucene.Net.Util;
@@ -21,6 +5,23 @@ using Spatial4n.Core.Shapes;
 
 namespace Lucene.Net.Spatial.Prefix
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// Implementation of
     /// <see cref="Lucene.Net.Spatial.Util.ShapeFieldCacheProvider{T}">Lucene.Net.Spatial.Util.ShapeFieldCacheProvider&lt;T&gt;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
index 20e611e..5176771 100644
--- a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
@@ -1,19 +1,3 @@
-/* 
- * 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.
- */
 #if !NET35
 using System.Collections.Concurrent;
 #else
@@ -32,6 +16,23 @@ using Spatial4n.Core.Shapes;
 
 namespace Lucene.Net.Spatial.Prefix
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// An abstract SpatialStrategy based on
     /// <see cref="Lucene.Net.Spatial.Prefix.Tree.SpatialPrefixTree">Lucene.Net.Spatial.Prefix.Tree.SpatialPrefixTree

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
index 198b671..1d49bf6 100644
--- a/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
@@ -1,20 +1,3 @@
-/* 
- * 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 Lucene.Net.Spatial.Prefix.Tree;
 using Lucene.Net.Spatial.Queries;
@@ -22,6 +5,23 @@ using Spatial4n.Core.Shapes;
 
 namespace Lucene.Net.Spatial.Prefix
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// A
     /// <see cref="PrefixTreeStrategy">PrefixTreeStrategy</see>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
index f7529ff..77afa1e 100644
--- a/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
@@ -1,19 +1,3 @@
-/* 
- * 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.Queries;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix.Tree;
@@ -24,6 +8,23 @@ using System.Collections.Generic;
 
 namespace Lucene.Net.Spatial.Prefix
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// A basic implementation of
     /// <see cref="PrefixTreeStrategy">PrefixTreeStrategy</see>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs b/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
index 49e6f0c..57a04ca 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
@@ -1,19 +1,3 @@
-/* 
- * 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 Spatial4n.Core.Shapes;
 using System;
 using System.Collections.Generic;
@@ -23,6 +7,23 @@ using System.Text;
 
 namespace Lucene.Net.Spatial.Prefix.Tree
 {
+    /*
+     * 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.
+     */
+
     /// <summary>Represents a grid cell.</summary>
     /// <remarks>
     /// Represents a grid cell. These are not necessarily thread-safe, although new

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
index 0dd43b3..a11f47e 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
@@ -1,19 +1,3 @@
-/* 
- * 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 Spatial4n.Core.Context;
 using Spatial4n.Core.Shapes;
 using Spatial4n.Core.Util;
@@ -22,6 +6,23 @@ using System.Collections.Generic;
 
 namespace Lucene.Net.Spatial.Prefix.Tree
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// A
     /// <see cref="SpatialPrefixTree">SpatialPrefixTree</see>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
index bd7ad5d..74dd43d 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
@@ -1,19 +1,3 @@
-/* 
- * 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 Spatial4n.Core.Context;
 using Spatial4n.Core.Shapes;
 using System;
@@ -24,6 +8,23 @@ using System.Text;
 
 namespace Lucene.Net.Spatial.Prefix.Tree
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// A
     /// <see cref="SpatialPrefixTree">SpatialPrefixTree</see>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
index bc71d92..9e5f2a3 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
@@ -1,19 +1,3 @@
-/* 
- * 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 Spatial4n.Core.Context;
 using Spatial4n.Core.Shapes;
 using System;
@@ -23,6 +7,23 @@ using System.Runtime.CompilerServices;
 
 namespace Lucene.Net.Spatial.Prefix.Tree
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// A spatial Prefix Tree, or Trie, which decomposes shapes into prefixed strings
     /// at variable lengths corresponding to variable precision.

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
index 1b6c26b..a1905ca 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
@@ -1,19 +1,3 @@
-/* 
- * 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 Spatial4n.Core.Context;
 using Spatial4n.Core.Distance;
 using System;
@@ -22,6 +6,23 @@ using System.Globalization;
 
 namespace Lucene.Net.Spatial.Prefix.Tree
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// Abstract Factory for creating
     /// <see cref="SpatialPrefixTree">SpatialPrefixTree</see>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
index 152e1b6..8021cd0 100644
--- a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
@@ -1,19 +1,3 @@
-/* 
- * 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.Index;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix.Tree;
@@ -27,6 +11,23 @@ using System.Diagnostics;
 
 namespace Lucene.Net.Spatial.Prefix
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// Finds docs where its indexed shape is
     /// <see cref="SpatialOperation.IsWithin">WITHIN</see>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialArgs.cs b/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
index 2668e84..d5921c8 100644
--- a/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
+++ b/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
@@ -1,26 +1,26 @@
-\ufeff/*
- * 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 Spatial4n.Core.Context;
+\ufeffusing Spatial4n.Core.Context;
 using Spatial4n.Core.Shapes;
 using System;
 
 namespace Lucene.Net.Spatial.Queries
 {
+    /*
+     * 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.
+     */
+
     public class SpatialArgs
     {
         public static readonly double DEFAULT_DISTERRPCT = 0.025d;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs b/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
index d743f31..280ce8e 100644
--- a/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
+++ b/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
@@ -1,21 +1,4 @@
-\ufeff/*
- * 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.Support;
+\ufeffusing Lucene.Net.Support;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Exceptions;
 using Spatial4n.Core.Shapes;
@@ -26,6 +9,23 @@ using System.Text;
 
 namespace Lucene.Net.Spatial.Queries
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// Parses a string that usually looks like "OPERATION(SHAPE)" into a {@link SpatialArgs}
     /// object. The set of operations supported are defined in {@link SpatialOperation}, such

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialOperation.cs b/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
index 407fdd4..ac1c9f3 100644
--- a/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
+++ b/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
@@ -1,19 +1,4 @@
-\ufeff/* See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * Esri Inc. 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 Spatial4n.Core.Shapes;
+\ufeffusing Spatial4n.Core.Shapes;
 using System;
 using System.Collections.Generic;
 using System.Globalization;
@@ -21,6 +6,23 @@ using System.Linq;
 
 namespace Lucene.Net.Spatial.Queries
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// A clause that compares a stored geometry to a supplied geometry. For more
     /// explanation of each operation, consider looking at the source implementation

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs b/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs
index e79027f..8d2a107 100644
--- a/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs
+++ b/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs
@@ -1,24 +1,24 @@
-\ufeff/*
- * 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;
+\ufeffusing System;
 
 namespace Lucene.Net.Spatial.Queries
 {
+    /*
+     * 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.
+     */
+
     [Serializable]
     public class UnsupportedSpatialOperation : NotSupportedException
     {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
index d95dc1e..ca383e9 100644
--- a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
+++ b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
@@ -1,21 +1,4 @@
-\ufeff/*
- * 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.Documents;
+\ufeffusing Lucene.Net.Documents;
 using Lucene.Net.Index;
 using Lucene.Net.Queries.Function;
 using Lucene.Net.Search;
@@ -32,6 +15,23 @@ using System.IO;
 
 namespace Lucene.Net.Spatial.Serialized
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// A SpatialStrategy based on serializing a Shape stored into BinaryDocValues.
     /// This is not at all fast; it's designed to be used in conjuction with another index based

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/SpatialStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/SpatialStrategy.cs b/src/Lucene.Net.Spatial/SpatialStrategy.cs
index a65ae0b..cc679e2 100644
--- a/src/Lucene.Net.Spatial/SpatialStrategy.cs
+++ b/src/Lucene.Net.Spatial/SpatialStrategy.cs
@@ -1,21 +1,4 @@
-\ufeff/*
- * 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.Documents;
+\ufeffusing Lucene.Net.Documents;
 using Lucene.Net.Queries.Function;
 using Lucene.Net.Queries.Function.ValueSources;
 using Lucene.Net.Search;
@@ -26,6 +9,23 @@ using System;
 
 namespace Lucene.Net.Spatial
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// The SpatialStrategy encapsulates an approach to indexing and searching based on shapes.
     /// <para/>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Util/CachingDoubleValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/CachingDoubleValueSource.cs b/src/Lucene.Net.Spatial/Util/CachingDoubleValueSource.cs
index 2047b52..c624a1b 100644
--- a/src/Lucene.Net.Spatial/Util/CachingDoubleValueSource.cs
+++ b/src/Lucene.Net.Spatial/Util/CachingDoubleValueSource.cs
@@ -1,21 +1,4 @@
-\ufeff/*
- * 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.Index;
+\ufeffusing Lucene.Net.Index;
 using Lucene.Net.Queries.Function;
 using Lucene.Net.Support;
 using System.Collections;
@@ -23,6 +6,23 @@ using System.Collections.Generic;
 
 namespace Lucene.Net.Spatial.Util
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// Caches the doubleVal of another value source in a HashMap
     /// so that it is computed only once.

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs b/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs
index cbb96c8..dc0a5d3 100644
--- a/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs
+++ b/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs
@@ -1,25 +1,25 @@
-\ufeff/*
- * 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 Spatial4n.Core.Shapes;
+\ufeffusing Spatial4n.Core.Shapes;
 using System.Collections.Generic;
 
 namespace Lucene.Net.Spatial.Util
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// Bounded Cache of Shapes associated with docIds.  Note, multiple Shapes can be
     /// associated with a given docId

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs b/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
index 660ea35..6df180f 100644
--- a/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
+++ b/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
@@ -1,21 +1,4 @@
-\ufeff/*
- * 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.Index;
+\ufeffusing Lucene.Net.Index;
 using Lucene.Net.Queries.Function;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Distance;
@@ -25,6 +8,23 @@ using System.Collections;
 
 namespace Lucene.Net.Spatial.Util
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// An implementation of the Lucene ValueSource that returns the spatial distance
     /// between an input point and a document's points in

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Util/ShapeFieldCacheProvider.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ShapeFieldCacheProvider.cs b/src/Lucene.Net.Spatial/Util/ShapeFieldCacheProvider.cs
index 45cb714..8e8dbc8 100644
--- a/src/Lucene.Net.Spatial/Util/ShapeFieldCacheProvider.cs
+++ b/src/Lucene.Net.Spatial/Util/ShapeFieldCacheProvider.cs
@@ -1,19 +1,3 @@
-/* 
- * 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.Index;
 using Lucene.Net.Search;
 using Lucene.Net.Support;
@@ -22,6 +6,23 @@ using Spatial4n.Core.Shapes;
 
 namespace Lucene.Net.Spatial.Util
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// Provides access to a
     /// <see cref="ShapeFieldCache{T}">ShapeFieldCache&lt;T&gt;</see>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs b/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
index 5ffc8a7..32a5bd9 100644
--- a/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
+++ b/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
@@ -1,21 +1,4 @@
-\ufeff/*
- * 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.Index;
+\ufeffusing Lucene.Net.Index;
 using Lucene.Net.Queries.Function;
 using Lucene.Net.Search;
 using Lucene.Net.Util;
@@ -23,6 +6,23 @@ using System;
 
 namespace Lucene.Net.Spatial.Util
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// Filter that matches all documents where a valuesource is
     /// in between a range of <c>min</c> and <c>max</c> inclusive.

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs b/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
index c456893..5a4cbd0 100644
--- a/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
+++ b/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
@@ -1,21 +1,4 @@
-\ufeff/*
- * 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.Index;
+\ufeffusing Lucene.Net.Index;
 using Lucene.Net.Queries.Function;
 using Lucene.Net.Search;
 using Lucene.Net.Util;
@@ -26,6 +9,23 @@ using System.Diagnostics;
 
 namespace Lucene.Net.Spatial.Vector
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// An implementation of the Lucene ValueSource model that returns the distance
     /// for a <see cref="PointVectorStrategy"/>.

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d21093a0/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
index 8c98261..19c9dc8 100644
--- a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
+++ b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
@@ -1,21 +1,4 @@
-\ufeff/*
- * 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.Documents;
+\ufeffusing Lucene.Net.Documents;
 using Lucene.Net.Queries.Function;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Queries;
@@ -26,6 +9,23 @@ using System;
 
 namespace Lucene.Net.Spatial.Vector
 {
+    /*
+     * 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.
+     */
+
     /// <summary>
     /// Simple <see cref="SpatialStrategy"/> which represents Points in two numeric <see cref="DoubleField"/>s.
     /// 


[06/26] lucenenet git commit: Changed namespace Lucene.Net.Query -> Lucene.Net.Queries to avoid naming conflicts with the Query class.

Posted by ni...@apache.org.
Changed namespace Lucene.Net.Query -> Lucene.Net.Queries to avoid naming conflicts with the Query class.


Project: http://git-wip-us.apache.org/repos/asf/lucenenet/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucenenet/commit/415cd5c0
Tree: http://git-wip-us.apache.org/repos/asf/lucenenet/tree/415cd5c0
Diff: http://git-wip-us.apache.org/repos/asf/lucenenet/diff/415cd5c0

Branch: refs/heads/master
Commit: 415cd5c07e45356ed85c31c86dac5c787505cdf2
Parents: d554792
Author: Shad Storhaug <sh...@shadstorhaug.com>
Authored: Wed Nov 16 00:18:01 2016 +0700
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Wed Nov 16 00:18:01 2016 +0700

----------------------------------------------------------------------
 src/Lucene.Net.Spatial/DisjointSpatialFilter.cs |   2 +-
 .../Prefix/ContainsPrefixTreeFilter.cs          |   2 +-
 .../Prefix/PrefixTreeStrategy.cs                |   2 +-
 .../Prefix/RecursivePrefixTreeStrategy.cs       |   4 +-
 .../Prefix/TermQueryPrefixTreeStrategy.cs       |   4 +-
 .../Prefix/Tree/SpatialPrefixTreeFactory.cs     |   4 +-
 .../Prefix/WithinPrefixTreeFilter.cs            |   2 +-
 src/Lucene.Net.Spatial/Query/SpatialArgs.cs     |   2 +-
 .../Query/SpatialArgsParser.cs                  |   2 +-
 .../Query/SpatialOperation.cs                   | 140 ++++++++++++++++---
 .../Query/UnsupportedSpatialOperation.cs        |   2 +-
 .../Serialized/SerializedDVStrategy.cs          |   2 +-
 src/Lucene.Net.Spatial/SpatialStrategy.cs       |   2 +-
 .../Vector/PointVectorStrategy.cs               |   2 +-
 14 files changed, 139 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/415cd5c0/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs b/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
index 3ad309e..e36a97b 100644
--- a/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
+++ b/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
@@ -18,7 +18,7 @@ using System;
 using Lucene.Net.Index;
 using Lucene.Net.Queries;
 using Lucene.Net.Search;
-using Lucene.Net.Spatial.Query;
+using Lucene.Net.Spatial.Queries;
 using Lucene.Net.Util;
 
 namespace Lucene.Net.Spatial

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/415cd5c0/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
index 017b518..76a95b8 100644
--- a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
@@ -22,7 +22,7 @@ using Lucene.Net.Index;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix;
 using Lucene.Net.Spatial.Prefix.Tree;
-using Lucene.Net.Spatial.Query;
+using Lucene.Net.Spatial.Queries;
 using Lucene.Net.Util;
 
 namespace Lucene.Net.Spatial.Prefix

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/415cd5c0/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
index 003b428..4a21ebc 100644
--- a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
@@ -27,7 +27,7 @@ using Lucene.Net.Documents;
 using Lucene.Net.Index;
 using Lucene.Net.Spatial.Prefix.Tree;
 using Lucene.Net.Queries.Function;
-using Lucene.Net.Spatial.Query;
+using Lucene.Net.Spatial.Queries;
 using Lucene.Net.Spatial.Util;
 using Lucene.Net.Support;
 using Spatial4n.Core.Shapes;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/415cd5c0/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
index 8740476..37b4d41 100644
--- a/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
@@ -18,7 +18,7 @@
 using Lucene.Net.Queries.Function;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix.Tree;
-using Lucene.Net.Spatial.Query;
+using Lucene.Net.Spatial.Queries;
 using Spatial4n.Core.Shapes;
 
 namespace Lucene.Net.Spatial.Prefix
@@ -78,7 +78,7 @@ namespace Lucene.Net.Spatial.Prefix
             {
                 return new DisjointSpatialFilter(this, args, FieldName);
             }
-            Shape shape = args.Shape;
+            IShape shape = args.Shape;
             int detailLevel = grid.GetLevelForDistance(args.ResolveDistErr(ctx, distErrPct));
 
         

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/415cd5c0/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
index aa67eeb..d4b5d25 100644
--- a/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
@@ -18,7 +18,7 @@ using System.Collections.Generic;
 using Lucene.Net.Queries;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix.Tree;
-using Lucene.Net.Spatial.Query;
+using Lucene.Net.Spatial.Queries;
 using Lucene.Net.Util;
 using Spatial4n.Core.Shapes;
 
@@ -56,7 +56,7 @@ namespace Lucene.Net.Spatial.Prefix
             {
                 throw new UnsupportedSpatialOperation(op);
             }
-            Shape shape = args.Shape;
+            IShape shape = args.Shape;
             int detailLevel = grid.GetLevelForDistance(args.ResolveDistErr(ctx, distErrPct));
             IList<Cell> cells = grid.GetCells(shape, detailLevel, false, true);
             //no parents

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/415cd5c0/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
index 1702e29..a1c7011 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
@@ -57,7 +57,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             string cname = args[PrefixTree];
             if (cname == null)
             {
-                cname = ctx.IsGeo() ? "geohash" : "quad";
+                cname = ctx.IsGeo ? "geohash" : "quad";
             }
             if ("geohash".Equals(cname, StringComparison.OrdinalIgnoreCase))
             {
@@ -106,7 +106,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             string maxDetailDistStr = args[MaxDistErr];
             if (maxDetailDistStr == null)
             {
-                if (!ctx.IsGeo())
+                if (!ctx.IsGeo)
                 {
                     return;
                 }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/415cd5c0/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
index 3081bbe..11cfc46 100644
--- a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
@@ -20,7 +20,7 @@ using System.Diagnostics;
 using Lucene.Net.Index;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix.Tree;
-using Lucene.Net.Spatial.Query;
+using Lucene.Net.Spatial.Queries;
 using Lucene.Net.Util;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Distance;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/415cd5c0/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialArgs.cs b/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
index 64e08aa..0ac68d2 100644
--- a/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
+++ b/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
@@ -19,7 +19,7 @@ using System;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Shapes;
 
-namespace Lucene.Net.Spatial.Query
+namespace Lucene.Net.Spatial.Queries
 {
     public class SpatialArgs
     {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/415cd5c0/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs b/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
index 6f0b433..8ffad19 100644
--- a/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
+++ b/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
@@ -21,7 +21,7 @@ using System.Text;
 using Lucene.Net.Support;
 using Spatial4n.Core.Context;
 
-namespace Lucene.Net.Spatial.Query
+namespace Lucene.Net.Spatial.Queries
 {
     public class SpatialArgsParser
     {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/415cd5c0/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialOperation.cs b/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
index 16c28a4..8d33ecf 100644
--- a/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
+++ b/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
@@ -13,17 +13,19 @@
  * limitations under the License.
  */
 
+using Spatial4n.Core.Shapes;
 using System;
 using System.Collections.Generic;
 using System.Globalization;
 using System.Linq;
 
-namespace Lucene.Net.Spatial.Query
+namespace Lucene.Net.Spatial.Queries
 {
-    public class SpatialOperation
+    [Serializable]
+    public abstract class SpatialOperation
     {
         // Private registry
-        private static readonly Dictionary<String, SpatialOperation> registry = new Dictionary<string, SpatialOperation>();
+        private static readonly Dictionary<string, SpatialOperation> registry = new Dictionary<string, SpatialOperation>();
         private static readonly IList<SpatialOperation> list = new List<SpatialOperation>();
 
         // Geometry Operations
@@ -31,27 +33,130 @@ namespace Lucene.Net.Spatial.Query
         /// <summary>
         /// Bounding box of the *indexed* shape.
         /// </summary>
-        public static readonly SpatialOperation BBoxIntersects = new SpatialOperation("BBoxIntersects", true, false, false);
+        public static readonly SpatialOperation BBoxIntersects = new BBoxIntersectsSpatialOperation();
+
+        private sealed class BBoxIntersectsSpatialOperation : SpatialOperation
+        {
+            internal BBoxIntersectsSpatialOperation()
+                : base("BBoxIntersects", true, false, false)
+            { }
+
+            public override bool Evaluate(IShape indexedShape, IShape queryShape)
+            {
+                return indexedShape.BoundingBox.Relate(queryShape).Intersects();
+            }
+        }
 
         /// <summary>
         /// Bounding box of the *indexed* shape.
         /// </summary>
-        public static readonly SpatialOperation BBoxWithin = new SpatialOperation("BBoxWithin", true, false, false);
+        public static readonly SpatialOperation BBoxWithin = new BBoxWithinSpatialOperation();
+
+        private sealed class BBoxWithinSpatialOperation : SpatialOperation
+        {
+            internal BBoxWithinSpatialOperation()
+                : base("BBoxWithin", true, false, false)
+            { }
+
+            public override bool Evaluate(IShape indexedShape, IShape queryShape)
+            {
+                IRectangle bbox = indexedShape.BoundingBox;
+                return bbox.Relate(queryShape) == SpatialRelation.WITHIN || bbox.Equals(queryShape);
+            }
+        }
+
+        public static readonly SpatialOperation Contains = new ContainsSpatialOperation();
+
+        private sealed class ContainsSpatialOperation : SpatialOperation
+        {
+            internal ContainsSpatialOperation()
+                : base("Contains", true, true, false)
+            { }
+
+            public override bool Evaluate(IShape indexedShape, IShape queryShape)
+            {
+                return indexedShape.HasArea && indexedShape.Relate(queryShape) == SpatialRelation.CONTAINS || indexedShape.Equals(queryShape);
+            }
+        }
+
+        public static readonly SpatialOperation Intersects = new IntersectsSpatialOperation();
+
+        private sealed class IntersectsSpatialOperation : SpatialOperation
+        {
+            internal IntersectsSpatialOperation()
+                : base("Intersects", true, false, false)
+            { }
+
+            public override bool Evaluate(IShape indexedShape, IShape queryShape)
+            {
+                return indexedShape.Relate(queryShape).Intersects();
+            }
+        }
+
+        public static readonly SpatialOperation IsEqualTo = new IsEqualToSpatialOperation();
+
+        private sealed class IsEqualToSpatialOperation : SpatialOperation
+        {
+            internal IsEqualToSpatialOperation()
+                : base("IsEqualTo", false, false, false)
+            { }
+
+            public override bool Evaluate(IShape indexedShape, IShape queryShape)
+            {
+                return indexedShape.Equals(queryShape);
+            }
+        }
+
+        public static readonly SpatialOperation IsDisjointTo = new IsDisjointToSpatialOperation();
+
+        private sealed class IsDisjointToSpatialOperation : SpatialOperation
+        {
+            internal IsDisjointToSpatialOperation()
+                : base("IsDisjointTo", false, false, false)
+            { }
+
+            public override bool Evaluate(IShape indexedShape, IShape queryShape)
+            {
+                return !indexedShape.Relate(queryShape).Intersects();
+            }
+        }
+
+        public static readonly SpatialOperation IsWithin = new IsWithinSpatialOperation();
+
+        private sealed class IsWithinSpatialOperation : SpatialOperation
+        {
+            internal IsWithinSpatialOperation()
+                : base("IsWithin", true, false, true)
+            { }
+
+            public override bool Evaluate(IShape indexedShape, IShape queryShape)
+            {
+                return queryShape.HasArea && (indexedShape.Relate(queryShape) == SpatialRelation.WITHIN || indexedShape.Equals(queryShape));
+            }
+        }
+
+        public static readonly SpatialOperation Overlaps = new OverlapsSpatialOperation();
+
+        private sealed class OverlapsSpatialOperation : SpatialOperation
+        {
+            internal OverlapsSpatialOperation()
+                : base("Overlaps", true, false, true)
+            { }
 
-        public static readonly SpatialOperation Contains = new SpatialOperation("Contains", true, true, false);
-        public static readonly SpatialOperation Intersects = new SpatialOperation("Intersects", true, false, false);
-        public static readonly SpatialOperation IsEqualTo = new SpatialOperation("IsEqualTo", false, false, false);
-        public static readonly SpatialOperation IsDisjointTo = new SpatialOperation("IsDisjointTo", false, false, false);
-        public static readonly SpatialOperation IsWithin = new SpatialOperation("IsWithin", true, false, true);
-        public static readonly SpatialOperation Overlaps = new SpatialOperation("Overlaps", true, false, true);
+            public override bool Evaluate(IShape indexedShape, IShape queryShape)
+            {
+                return queryShape.HasArea && indexedShape.Relate(queryShape).Intersects();
+            }
+        }
 
         // Member variables
         private readonly bool scoreIsMeaningful;
         private readonly bool sourceNeedsArea;
         private readonly bool targetNeedsArea;
-        private readonly String name;
+        private readonly string name;
 
-        protected SpatialOperation(String name, bool scoreIsMeaningful, bool sourceNeedsArea, bool targetNeedsArea)
+
+        protected SpatialOperation(string name, bool scoreIsMeaningful, bool sourceNeedsArea, bool targetNeedsArea)
         {
             this.name = name;
             this.scoreIsMeaningful = scoreIsMeaningful;
@@ -62,7 +167,7 @@ namespace Lucene.Net.Spatial.Query
             list.Add(this);
         }
 
-        public static SpatialOperation Get(String v)
+        public static SpatialOperation Get(string v)
         {
             SpatialOperation op;
             if (!registry.TryGetValue(v, out op) || op == null)
@@ -83,6 +188,8 @@ namespace Lucene.Net.Spatial.Query
             return tst.Any(t => op == t);
         }
 
+        public abstract bool Evaluate(IShape indexedShape, IShape queryShape);
+
         // ================================================= Getters / Setters =============================================
 
         public bool IsScoreIsMeaningful()
@@ -100,15 +207,14 @@ namespace Lucene.Net.Spatial.Query
             return targetNeedsArea;
         }
 
-        public String Name
+        public string Name
         {
             get { return name; }
         }
 
-        public override String ToString()
+        public override string ToString()
         {
             return name;
         }
-
     }
 }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/415cd5c0/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs b/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs
index 92ae852..35949bc 100644
--- a/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs
+++ b/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs
@@ -17,7 +17,7 @@
 
 using System;
 
-namespace Lucene.Net.Spatial.Query
+namespace Lucene.Net.Spatial.Queries
 {
     [Serializable]
     public class UnsupportedSpatialOperation : InvalidOperationException

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/415cd5c0/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
index c0692d6..92e4aaf 100644
--- a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
+++ b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
@@ -25,7 +25,7 @@ using Lucene.Net.Documents;
 using Lucene.Net.Queries.Function;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial;
-using Lucene.Net.Spatial.Query;
+using Lucene.Net.Spatial.Queries;
 using Lucene.Net.Support;
 using Lucene.Net.Util;
 using Spatial4n.Core.Context;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/415cd5c0/src/Lucene.Net.Spatial/SpatialStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/SpatialStrategy.cs b/src/Lucene.Net.Spatial/SpatialStrategy.cs
index f2de112..5f16c71 100644
--- a/src/Lucene.Net.Spatial/SpatialStrategy.cs
+++ b/src/Lucene.Net.Spatial/SpatialStrategy.cs
@@ -20,7 +20,7 @@ using Lucene.Net.Documents;
 using Lucene.Net.Queries.Function;
 using Lucene.Net.Queries.Function.ValueSources;
 using Lucene.Net.Search;
-using Lucene.Net.Spatial.Query;
+using Lucene.Net.Spatial.Queries;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Shapes;
 

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/415cd5c0/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
index 25672b9..5c0d264 100644
--- a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
+++ b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
@@ -19,7 +19,7 @@ using System;
 using Lucene.Net.Documents;
 using Lucene.Net.Queries.Function;
 using Lucene.Net.Search;
-using Lucene.Net.Spatial.Query;
+using Lucene.Net.Spatial.Queries;
 using Lucene.Net.Spatial.Util;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Shapes;


[22/26] lucenenet git commit: Spatial: Removed commented code that helped with the port

Posted by ni...@apache.org.
Spatial: Removed commented code that helped with the port


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

Branch: refs/heads/master
Commit: aa35405f94b5ffeef2357aa8e85b19fffa9d9b9b
Parents: d21093a
Author: Shad Storhaug <sh...@shadstorhaug.com>
Authored: Fri Nov 18 00:10:21 2016 +0700
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Fri Nov 18 01:38:31 2016 +0700

----------------------------------------------------------------------
 .../Prefix/AbstractVisitingPrefixTreeFilter.cs  |   2 -
 .../Prefix/Tree/QuadPrefixTree.cs               |   4 -
 .../Serialized/SerializedDVStrategy.cs          | 115 -------------------
 .../Util/DistanceToShapeValueSource.cs          |  21 ----
 .../Util/ShapeFieldCacheProvider.cs             |   5 -
 .../Util/ShapePredicateValueSource.cs           |  20 ----
 .../QueryEqualsHashCodeTest.cs                  |  23 ----
 7 files changed, 190 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/aa35405f/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
index 42c76d1..1286158 100644
--- a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
@@ -435,8 +435,6 @@ namespace Lucene.Net.Spatial.Prefix
                 {
                     get
                     {
-                        //Debug.Assert(cellIter.Current != null);
-                        //vNode.Reset(cellIter.Current);
                         return vNode;
                     }
                 }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/aa35405f/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
index 74dd43d..1222d27 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
@@ -121,10 +121,6 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         {
             // Format the number to min 3 integer digits and exactly 5 fraction digits
             const string FORMAT_STR = @"000.00000";
-            /*NumberFormat nf = NumberFormat.GetNumberInstance(CultureInfo.Root);
-			nf.SetMaximumFractionDigits(5);
-			nf.SetMinimumFractionDigits(5);
-			nf.SetMinimumIntegerDigits(3);*/
             for (int i = 0; i < maxLevels; i++)
             {
                 @out.WriteLine(i + "]\t" + levelW[i].ToString(FORMAT_STR) + "\t" + levelH[i].ToString(FORMAT_STR) + "\t" +

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/aa35405f/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
index ca383e9..33926ee 100644
--- a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
+++ b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
@@ -69,17 +69,6 @@ namespace Lucene.Net.Spatial.Serialized
 
                 //this is a hack to avoid redundant byte array copying by byteStream.toByteArray()
                 byteStream.WriteTo(new OutputStreamAnonymousHelper(bytesRef));
-
-
-                //            byteStream.WriteTo(new FilterOutputStream(null/*not used*/) {
-                //    @Override
-                //    public void write(byte[] b, int off, int len) throws IOException
-                //    {
-                //        bytesRef.bytes = b;
-                //        bytesRef.offset = off;
-                //        bytesRef.length = len;
-                //    }
-                //});
             }
             catch (IOException e)
             {
@@ -157,42 +146,6 @@ namespace Lucene.Net.Spatial.Serialized
             public override DocIdSet GetDocIdSet(AtomicReaderContext context, Bits acceptDocs)
             {
                 return new DocIdSetAnonymousHelper(this, context, acceptDocs);
-
-                //      return new DocIdSet()
-                //        {
-                //            @Override
-                //        public DocIdSetIterator iterator() throws IOException
-                //        {
-                //          throw new UnsupportedOperationException(
-                //              "Iteration is too slow; instead try FilteredQuery.QUERY_FIRST_FILTER_STRATEGY");
-                //        //Note that if you're truly bent on doing this, then see FunctionValues.getRangeScorer
-                //    }
-
-                //    @Override
-                //        public Bits bits() throws IOException
-                //    {
-                //        //null Map context -- we simply don't have one. That's ok.
-                //        final FunctionValues predFuncValues = predicateValueSource.getValues(null, context);
-
-                //          return new Bits()
-                //    {
-
-                //        @Override
-                //            public boolean get(int index)
-                //    {
-                //        if (acceptDocs != null && !acceptDocs.get(index))
-                //            return false;
-                //        return predFuncValues.boolVal(index);
-                //    }
-
-                //    @Override
-                //            public int length()
-                //    {
-                //        return context.reader().maxDoc();
-                //    }
-                //};
-                //  }
-                //};
             }
 
             internal class DocIdSetAnonymousHelper : DocIdSet
@@ -293,74 +246,6 @@ namespace Lucene.Net.Spatial.Serialized
                 BinaryDocValues docValues = readerContext.AtomicReader.GetBinaryDocValues(fieldName);
 
                 return new FuctionValuesAnonymousHelper(this, docValues);
-
-                //      return new FunctionValues()
-                //{
-                //    int bytesRefDoc = -1;
-                //    BytesRef bytesRef = new BytesRef();//scratch
-
-                //    bool fillBytes(int doc) {
-                //        if (bytesRefDoc != doc)
-                //        {
-                //            docValues.Get(doc, bytesRef);
-                //            bytesRefDoc = doc;
-                //        }
-                //        return bytesRef.length != 0;
-                //    }
-
-                //    @Override
-                //        public boolean exists(int doc)
-                //{
-                //    return fillBytes(doc);
-                //}
-
-                //@Override
-                //        public boolean bytesVal(int doc, BytesRef target)
-                //{
-                //    if (fillBytes(doc))
-                //    {
-                //        target.bytes = bytesRef.bytes;
-                //        target.offset = bytesRef.offset;
-                //        target.length = bytesRef.length;
-                //        return true;
-                //    }
-                //    else
-                //    {
-                //        target.length = 0;
-                //        return false;
-                //    }
-                //}
-
-                //@Override
-                //        public Object objectVal(int docId)
-                //{
-                //    if (!fillBytes(docId))
-                //        return null;
-                //    DataInputStream dataInput = new DataInputStream(
-                //        new ByteArrayInputStream(bytesRef.bytes, bytesRef.offset, bytesRef.length));
-                //    try
-                //    {
-                //        return binaryCodec.readShape(dataInput);
-                //    }
-                //    catch (IOException e)
-                //    {
-                //        throw new RuntimeException(e);
-                //    }
-                //}
-
-                //@Override
-                //        public Explanation explain(int doc)
-                //{
-                //    return new Explanation(Float.NaN, toString(doc));
-                //}
-
-                //@Override
-                //        public String toString(int doc)
-                //{
-                //    return description() + "=" + objectVal(doc);//TODO truncate?
-                //}
-
-                //      };
             }
 
             internal class FuctionValuesAnonymousHelper : FunctionValues

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/aa35405f/src/Lucene.Net.Spatial/Util/DistanceToShapeValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/DistanceToShapeValueSource.cs b/src/Lucene.Net.Spatial/Util/DistanceToShapeValueSource.cs
index 76fc7bd..c108e54 100644
--- a/src/Lucene.Net.Spatial/Util/DistanceToShapeValueSource.cs
+++ b/src/Lucene.Net.Spatial/Util/DistanceToShapeValueSource.cs
@@ -70,27 +70,6 @@ namespace Lucene.Net.Spatial.Util
             FunctionValues shapeValues = shapeValueSource.GetValues(context, readerContext);
 
             return new DoubleDocValuesAnonymousHelper(this, shapeValues);
-
-            //return new DoubleDocValues(this)
-            //    {
-            //        @Override
-            //  public double doubleVal(int doc)
-            //    {
-            //        Shape shape = (Shape)shapeValues.objectVal(doc);
-            //        if (shape == null || shape.isEmpty())
-            //            return nullValue;
-            //        Point pt = shape.getCenter();
-            //        return distCalc.distance(queryPoint, pt) * multiplier;
-            //    }
-
-            //    @Override
-            //  public Explanation explain(int doc)
-            //    {
-            //        Explanation exp = super.explain(doc);
-            //        exp.addDetail(shapeValues.explain(doc));
-            //        return exp;
-            //    }
-            //};
         }
 
         internal class DoubleDocValuesAnonymousHelper : DoubleDocValues

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/aa35405f/src/Lucene.Net.Spatial/Util/ShapeFieldCacheProvider.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ShapeFieldCacheProvider.cs b/src/Lucene.Net.Spatial/Util/ShapeFieldCacheProvider.cs
index 8e8dbc8..4fba809 100644
--- a/src/Lucene.Net.Spatial/Util/ShapeFieldCacheProvider.cs
+++ b/src/Lucene.Net.Spatial/Util/ShapeFieldCacheProvider.cs
@@ -39,13 +39,8 @@ namespace Lucene.Net.Spatial.Util
     {
         //private Logger log = Logger.GetLogger(GetType().FullName);
 
-#if !NET35
         private readonly WeakDictionary<IndexReader, ShapeFieldCache<T>> sidx =
             new WeakDictionary<IndexReader, ShapeFieldCache<T>>();
-#else
-	    private readonly WeakDictionary<IndexReader, ShapeFieldCache<T>> sidx =
-	        new WeakDictionary<IndexReader, ShapeFieldCache<T>>();
-#endif
 
         protected internal readonly int defaultSize;
         protected internal readonly string shapeField;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/aa35405f/src/Lucene.Net.Spatial/Util/ShapePredicateValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ShapePredicateValueSource.cs b/src/Lucene.Net.Spatial/Util/ShapePredicateValueSource.cs
index 7f65893..8914343 100644
--- a/src/Lucene.Net.Spatial/Util/ShapePredicateValueSource.cs
+++ b/src/Lucene.Net.Spatial/Util/ShapePredicateValueSource.cs
@@ -70,26 +70,6 @@ namespace Lucene.Net.Spatial.Util
             FunctionValues shapeValues = shapeValuesource.GetValues(context, readerContext);
 
             return new BoolDocValuesAnonymousHelper(this, shapeValues);
-
-            //return new BoolDocValues(this)
-            //    {
-            //        @Override
-            //  public boolean boolVal(int doc)
-            //    {
-            //        Shape indexedShape = (Shape)shapeValues.objectVal(doc);
-            //        if (indexedShape == null)
-            //            return false;
-            //        return op.evaluate(indexedShape, queryShape);
-            //    }
-
-            //    @Override
-            //  public Explanation explain(int doc)
-            //    {
-            //        Explanation exp = super.explain(doc);
-            //        exp.addDetail(shapeValues.explain(doc));
-            //        return exp;
-            //    }
-            //};
         }
 
         internal class BoolDocValuesAnonymousHelper : BoolDocValues

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/aa35405f/src/Lucene.Net.Tests.Spatial/QueryEqualsHashCodeTest.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/QueryEqualsHashCodeTest.cs b/src/Lucene.Net.Tests.Spatial/QueryEqualsHashCodeTest.cs
index 25db926..33417b5 100644
--- a/src/Lucene.Net.Tests.Spatial/QueryEqualsHashCodeTest.cs
+++ b/src/Lucene.Net.Tests.Spatial/QueryEqualsHashCodeTest.cs
@@ -102,31 +102,8 @@ namespace Lucene.Net.Spatial
             SpatialArgs args1 = MakeArgs1();
             SpatialArgs args2 = MakeArgs2();
             TestEqualsHashcode(args1, args2, new ObjGeneratorQueryAnonymousHelper(strategy));
-            //        testEqualsHashcode(args1, args2, new ObjGenerator() {
-            //  @Override
-            //  public Object gen(SpatialArgs args)
-            //    {
-            //        return strategy.makeQuery(args);
-            //    }
-            //});
             TestEqualsHashcode(args1, args2, new ObjGeneratorFilterAnonymousHelper(strategy));
-            //    testEqualsHashcode(args1, args2, new ObjGenerator()
-            //    {
-            //        @Override
-            //      public Object gen(SpatialArgs args)
-            //    {
-            //        return strategy.makeFilter(args);
-            //    }
-            //});
             TestEqualsHashcode(args1, args2, new ObjGeneratorDistanceValueSourceAnonymousHelper(strategy));
-            //            testEqualsHashcode(args1, args2, new ObjGenerator()
-            //{
-            //    @Override
-            //      public Object gen(SpatialArgs args)
-            //{
-            //    return strategy.makeDistanceValueSource(args.getShape().getCenter());
-            //}
-            //    });
         }
 
         private void TestEqualsHashcode(SpatialArgs args1, SpatialArgs args2, ObjGenerator generator)


[20/26] lucenenet git commit: Spatial: Removed and sorted usings

Posted by ni...@apache.org.
Spatial: Removed and sorted usings


Project: http://git-wip-us.apache.org/repos/asf/lucenenet/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucenenet/commit/553851c2
Tree: http://git-wip-us.apache.org/repos/asf/lucenenet/tree/553851c2
Diff: http://git-wip-us.apache.org/repos/asf/lucenenet/diff/553851c2

Branch: refs/heads/master
Commit: 553851c2e05977287f75ed7577b6f2639416a778
Parents: a0b447f
Author: Shad Storhaug <sh...@shadstorhaug.com>
Authored: Thu Nov 17 23:44:57 2016 +0700
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Fri Nov 18 01:37:07 2016 +0700

----------------------------------------------------------------------
 src/Lucene.Net.Spatial/DisjointSpatialFilter.cs      |  2 +-
 .../Prefix/AbstractPrefixTreeFilter.cs               |  2 +-
 .../Prefix/AbstractVisitingPrefixTreeFilter.cs       |  9 ++++-----
 .../Prefix/ContainsPrefixTreeFilter.cs               |  9 +++------
 .../Prefix/PointPrefixTreeFieldCacheProvider.cs      |  2 --
 src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs  |  2 --
 .../Prefix/RecursivePrefixTreeStrategy.cs            |  1 -
 .../Prefix/TermQueryPrefixTreeStrategy.cs            |  2 +-
 src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs           |  5 +----
 .../Prefix/Tree/GeohashPrefixTree.cs                 |  4 ++--
 src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs |  4 ++--
 .../Prefix/Tree/SpatialPrefixTree.cs                 |  7 ++-----
 .../Prefix/Tree/SpatialPrefixTreeFactory.cs          |  4 ++--
 .../Prefix/WithinPrefixTreeFilter.cs                 |  7 +++----
 src/Lucene.Net.Spatial/Query/SpatialArgs.cs          |  2 +-
 src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs    |  8 ++++----
 src/Lucene.Net.Spatial/Query/SpatialOperation.cs     |  1 -
 .../Serialized/SerializedDVStrategy.cs               | 15 +++++----------
 src/Lucene.Net.Spatial/SpatialStrategy.cs            |  2 +-
 .../Util/CachingDoubleValueSource.cs                 |  4 ++--
 src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs       |  2 +-
 .../Util/ShapeFieldCacheDistanceValueSource.cs       |  5 ++---
 src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs     |  2 +-
 src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs |  5 ++---
 src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs |  2 +-
 25 files changed, 42 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs b/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
index 6f2fdd9..9c0235e 100644
--- a/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
+++ b/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-using System;
 using Lucene.Net.Index;
 using Lucene.Net.Queries;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Queries;
 using Lucene.Net.Util;
+using System;
 
 namespace Lucene.Net.Spatial
 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
index 666642a..08c5e4c 100644
--- a/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-using System.Diagnostics;
 using Lucene.Net.Index;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix.Tree;
 using Lucene.Net.Util;
 using Spatial4n.Core.Shapes;
+using System.Diagnostics;
 
 namespace Lucene.Net.Spatial.Prefix
 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
index d3aa5df..af8920d 100644
--- a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
@@ -14,16 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-using System;
-using System.Collections;
-using System.Collections.Generic;
-using System.Diagnostics;
 using Lucene.Net.Index;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix.Tree;
-using Lucene.Net.Spatial.Util;
 using Lucene.Net.Util;
 using Spatial4n.Core.Shapes;
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Diagnostics;
 
 namespace Lucene.Net.Spatial.Prefix
 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
index 466cab8..0c244cc 100644
--- a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
@@ -14,16 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-using System;
-using System.Collections.Generic;
-using Lucene.Net.Spatial.Util;
-using Spatial4n.Core.Shapes;
 using Lucene.Net.Index;
 using Lucene.Net.Search;
-using Lucene.Net.Spatial.Prefix;
 using Lucene.Net.Spatial.Prefix.Tree;
-using Lucene.Net.Spatial.Queries;
 using Lucene.Net.Util;
+using Spatial4n.Core.Shapes;
+using System;
+using System.Collections.Generic;
 using System.Diagnostics;
 
 namespace Lucene.Net.Spatial.Prefix

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs b/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
index db36767..e4cee6a 100644
--- a/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
+++ b/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
@@ -14,8 +14,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-using System;
-using Lucene.Net.Index;
 using Lucene.Net.Spatial.Prefix.Tree;
 using Lucene.Net.Spatial.Util;
 using Lucene.Net.Util;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
index a10e717..20e611e 100644
--- a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
@@ -14,7 +14,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-using System;
 #if !NET35
 using System.Collections.Concurrent;
 #else
@@ -29,7 +28,6 @@ using Lucene.Net.Spatial.Prefix.Tree;
 using Lucene.Net.Queries.Function;
 using Lucene.Net.Spatial.Queries;
 using Lucene.Net.Spatial.Util;
-using Lucene.Net.Support;
 using Spatial4n.Core.Shapes;
 
 namespace Lucene.Net.Spatial.Prefix

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
index 41ae1d0..198b671 100644
--- a/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
@@ -15,7 +15,6 @@
  * limitations under the License.
  */
 
-using Lucene.Net.Queries.Function;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix.Tree;
 using Lucene.Net.Spatial.Queries;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
index 8017eb3..f7529ff 100644
--- a/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-using System.Collections.Generic;
 using Lucene.Net.Queries;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix.Tree;
 using Lucene.Net.Spatial.Queries;
 using Lucene.Net.Util;
 using Spatial4n.Core.Shapes;
+using System.Collections.Generic;
 
 namespace Lucene.Net.Spatial.Prefix
 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs b/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
index 5969a08..49e6f0c 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
@@ -14,15 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+using Spatial4n.Core.Shapes;
 using System;
 using System.Collections.Generic;
 using System.Diagnostics;
 using System.Runtime.CompilerServices;
 using System.Text;
-using Lucene.Net.Spatial.Util;
-using Lucene.Net.Util;
-using Spatial4n.Core.Shapes;
-using Spatial4n.Core.Context;
 
 namespace Lucene.Net.Spatial.Prefix.Tree
 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
index 2c775c8..0dd43b3 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-using System;
-using System.Collections.Generic;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Shapes;
 using Spatial4n.Core.Util;
+using System;
+using System.Collections.Generic;
 
 namespace Lucene.Net.Spatial.Prefix.Tree
 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
index ce6f48b..bd7ad5d 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+using Spatial4n.Core.Context;
+using Spatial4n.Core.Shapes;
 using System;
 using System.Collections.Generic;
 using System.Diagnostics;
 using System.IO;
 using System.Text;
-using Spatial4n.Core.Context;
-using Spatial4n.Core.Shapes;
 
 namespace Lucene.Net.Spatial.Prefix.Tree
 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
index 9964485..bc71d92 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
@@ -14,15 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+using Spatial4n.Core.Context;
+using Spatial4n.Core.Shapes;
 using System;
 using System.Collections.Generic;
-using System.Collections.ObjectModel;
 using System.Diagnostics;
-using System.Linq;
 using System.Runtime.CompilerServices;
-using System.Text;
-using Spatial4n.Core.Context;
-using Spatial4n.Core.Shapes;
 
 namespace Lucene.Net.Spatial.Prefix.Tree
 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
index ad70563..1b6c26b 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-using System;
-using System.Collections.Generic;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Distance;
+using System;
+using System.Collections.Generic;
 using System.Globalization;
 
 namespace Lucene.Net.Spatial.Prefix.Tree

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
index 110739f..152e1b6 100644
--- a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
@@ -14,17 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-using System;
-using System.Collections.Generic;
-using System.Diagnostics;
 using Lucene.Net.Index;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix.Tree;
-using Lucene.Net.Spatial.Queries;
 using Lucene.Net.Util;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Distance;
 using Spatial4n.Core.Shapes;
+using System;
+using System.Collections.Generic;
+using System.Diagnostics;
 
 namespace Lucene.Net.Spatial.Prefix
 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialArgs.cs b/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
index 83acc37..2668e84 100644
--- a/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
+++ b/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-using System;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Shapes;
+using System;
 
 namespace Lucene.Net.Spatial.Queries
 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs b/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
index e71c9b5..d743f31 100644
--- a/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
+++ b/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-using System;
-using System.Collections.Generic;
-using System.Text;
 using Lucene.Net.Support;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Exceptions;
-using System.Linq;
 using Spatial4n.Core.Shapes;
+using System;
+using System.Collections.Generic;
+using System.Linq;
+using System.Text;
 
 namespace Lucene.Net.Spatial.Queries
 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialOperation.cs b/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
index 5f5aa35..407fdd4 100644
--- a/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
+++ b/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
@@ -13,7 +13,6 @@
  * limitations under the License.
  */
 
-using Lucene.Net.Support;
 using Spatial4n.Core.Shapes;
 using System;
 using System.Collections.Generic;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
index 2358f19..d95dc1e 100644
--- a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
+++ b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
@@ -15,25 +15,20 @@
  * limitations under the License.
  */
 
-using System;
-using System.Collections.Generic;
-using System.IO;
-using System.Linq;
-using System.Text;
-using System.Threading.Tasks;
 using Lucene.Net.Documents;
+using Lucene.Net.Index;
 using Lucene.Net.Queries.Function;
 using Lucene.Net.Search;
-using Lucene.Net.Spatial;
 using Lucene.Net.Spatial.Queries;
+using Lucene.Net.Spatial.Util;
 using Lucene.Net.Support;
 using Lucene.Net.Util;
 using Spatial4n.Core.Context;
-using Spatial4n.Core.Shapes;
-using Lucene.Net.Index;
 using Spatial4n.Core.Io;
+using Spatial4n.Core.Shapes;
+using System;
 using System.Collections;
-using Lucene.Net.Spatial.Util;
+using System.IO;
 
 namespace Lucene.Net.Spatial.Serialized
 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/SpatialStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/SpatialStrategy.cs b/src/Lucene.Net.Spatial/SpatialStrategy.cs
index 3c80f68..a65ae0b 100644
--- a/src/Lucene.Net.Spatial/SpatialStrategy.cs
+++ b/src/Lucene.Net.Spatial/SpatialStrategy.cs
@@ -15,7 +15,6 @@
  * limitations under the License.
  */
 
-using System;
 using Lucene.Net.Documents;
 using Lucene.Net.Queries.Function;
 using Lucene.Net.Queries.Function.ValueSources;
@@ -23,6 +22,7 @@ using Lucene.Net.Search;
 using Lucene.Net.Spatial.Queries;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Shapes;
+using System;
 
 namespace Lucene.Net.Spatial
 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Util/CachingDoubleValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/CachingDoubleValueSource.cs b/src/Lucene.Net.Spatial/Util/CachingDoubleValueSource.cs
index a68ed73..2047b52 100644
--- a/src/Lucene.Net.Spatial/Util/CachingDoubleValueSource.cs
+++ b/src/Lucene.Net.Spatial/Util/CachingDoubleValueSource.cs
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-using System.Collections;
-using System.Collections.Generic;
 using Lucene.Net.Index;
 using Lucene.Net.Queries.Function;
 using Lucene.Net.Support;
+using System.Collections;
+using System.Collections.Generic;
 
 namespace Lucene.Net.Spatial.Util
 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs b/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs
index 96cfb3e..cbb96c8 100644
--- a/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs
+++ b/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs
@@ -15,8 +15,8 @@
  * limitations under the License.
  */
 
-using System.Collections.Generic;
 using Spatial4n.Core.Shapes;
+using System.Collections.Generic;
 
 namespace Lucene.Net.Spatial.Util
 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs b/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
index 144424e..660ea35 100644
--- a/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
+++ b/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
@@ -15,14 +15,13 @@
  * limitations under the License.
  */
 
-using System;
-using System.Collections;
-using System.Collections.Generic;
 using Lucene.Net.Index;
 using Lucene.Net.Queries.Function;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Distance;
 using Spatial4n.Core.Shapes;
+using System;
+using System.Collections;
 
 namespace Lucene.Net.Spatial.Util
 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs b/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
index 45ffdd6..5ffc8a7 100644
--- a/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
+++ b/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-using System;
 using Lucene.Net.Index;
 using Lucene.Net.Queries.Function;
 using Lucene.Net.Search;
 using Lucene.Net.Util;
+using System;
 
 namespace Lucene.Net.Spatial.Util
 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs b/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
index a186558..c456893 100644
--- a/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
+++ b/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
@@ -15,15 +15,14 @@
  * limitations under the License.
  */
 
-using System.Collections;
-using System.Collections.Generic;
-using System.Diagnostics;
 using Lucene.Net.Index;
 using Lucene.Net.Queries.Function;
 using Lucene.Net.Search;
 using Lucene.Net.Util;
 using Spatial4n.Core.Distance;
 using Spatial4n.Core.Shapes;
+using System.Collections;
+using System.Diagnostics;
 
 namespace Lucene.Net.Spatial.Vector
 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/553851c2/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
index d46bff0..8c98261 100644
--- a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
+++ b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
@@ -15,7 +15,6 @@
  * limitations under the License.
  */
 
-using System;
 using Lucene.Net.Documents;
 using Lucene.Net.Queries.Function;
 using Lucene.Net.Search;
@@ -23,6 +22,7 @@ using Lucene.Net.Spatial.Queries;
 using Lucene.Net.Spatial.Util;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Shapes;
+using System;
 
 namespace Lucene.Net.Spatial.Vector
 {


[18/26] lucenenet git commit: Repaired several issues that were causing tests to fail. Refactored some of the API, added some missing documentation, and set member accessibility on some members.

Posted by ni...@apache.org.
Repaired several issues that were causing tests to fail. Refactored some of the API, added some missing documentation, and set member accessibility on some members.


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

Branch: refs/heads/master
Commit: df3f64d718a58dfa43943b9e7ddf25e6e118a4d5
Parents: d8c7353
Author: Shad Storhaug <sh...@shadstorhaug.com>
Authored: Wed Nov 16 20:57:17 2016 +0700
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Thu Nov 17 22:14:19 2016 +0700

----------------------------------------------------------------------
 src/Lucene.Net.Core/Search/DocIdSetIterator.cs  |   2 +-
 src/Lucene.Net.Core/Util/Bits.cs                |   2 +-
 src/Lucene.Net.Spatial/DisjointSpatialFilter.cs |   8 +-
 .../Prefix/AbstractPrefixTreeFilter.cs          |  24 ++-
 .../Prefix/AbstractVisitingPrefixTreeFilter.cs  | 160 ++++++++--------
 .../Prefix/ContainsPrefixTreeFilter.cs          |  81 ++++----
 .../Prefix/IntersectsPrefixTreeFilter.cs        |   3 -
 .../Prefix/PrefixTreeStrategy.cs                |  11 +-
 .../Prefix/RecursivePrefixTreeStrategy.cs       |   5 +-
 .../Prefix/TermQueryPrefixTreeStrategy.cs       |   7 +-
 src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs      |  54 ++----
 .../Prefix/Tree/GeohashPrefixTree.cs            |  75 ++++----
 .../Prefix/Tree/QuadPrefixTree.cs               | 183 +++++++++----------
 .../Prefix/Tree/SpatialPrefixTree.cs            |  22 +--
 .../Prefix/Tree/SpatialPrefixTreeFactory.cs     |  47 ++---
 .../Prefix/WithinPrefixTreeFilter.cs            |  81 ++++----
 src/Lucene.Net.Spatial/Query/SpatialArgs.cs     |  40 ++--
 .../Query/SpatialArgsParser.cs                  | 101 ++++++----
 .../Query/SpatialOperation.cs                   |  39 ++--
 .../Query/UnsupportedSpatialOperation.cs        |   2 +-
 .../Serialized/SerializedDVStrategy.cs          |  20 +-
 src/Lucene.Net.Spatial/SpatialStrategy.cs       |  31 +++-
 .../Util/CachingDoubleValueSource.cs            |  40 ++--
 .../Util/DistanceToShapeValueSource.cs          |  56 +++---
 src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs  |  10 +-
 .../Util/ShapeFieldCacheDistanceValueSource.cs  |  36 ++--
 .../Util/ShapeFieldCacheProvider.cs             |  21 +--
 .../Util/ShapePredicateValueSource.cs           |  55 +++---
 .../Util/ValueSourceFilter.cs                   |  16 +-
 .../Vector/DistanceValueSource.cs               |  80 ++++----
 .../Vector/PointVectorStrategy.cs               | 119 ++++++------
 .../Prefix/SpatialOpRecursivePrefixTreeTest.cs  |   1 +
 32 files changed, 743 insertions(+), 689 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Core/Search/DocIdSetIterator.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Core/Search/DocIdSetIterator.cs b/src/Lucene.Net.Core/Search/DocIdSetIterator.cs
index 4ec0fec..b89b10f 100644
--- a/src/Lucene.Net.Core/Search/DocIdSetIterator.cs
+++ b/src/Lucene.Net.Core/Search/DocIdSetIterator.cs
@@ -87,7 +87,7 @@ namespace Lucene.Net.Search
         ///
         /// @since 2.9
         /// </summary>
-        public abstract int DocID();
+        public abstract int DocID(); // LUCENENET TODO: Change to property getter
 
         /// <summary>
         /// Advances to the next document in the set and returns the doc it is

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Core/Util/Bits.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Core/Util/Bits.cs b/src/Lucene.Net.Core/Util/Bits.cs
index ac60e78..26866e2 100644
--- a/src/Lucene.Net.Core/Util/Bits.cs
+++ b/src/Lucene.Net.Core/Util/Bits.cs
@@ -34,7 +34,7 @@ namespace Lucene.Net.Util
 
         /// <summary>
         /// Returns the number of bits in this set </summary>
-        int Length();
+        int Length(); // LUCENENET TODO: Change to property getter
     }
 
     public static class BitsHelpers

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs b/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
index 41842af..6f2fdd9 100644
--- a/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
+++ b/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
@@ -48,15 +48,13 @@ namespace Lucene.Net.Spatial
         /// <param name="args">Used in spatial intersection</param>
         /// <param name="field">
         /// This field is used to determine which docs have spatial data via
-        /// <see cref="Org.Apache.Lucene.Search.FieldCache.GetDocsWithField(Org.Apache.Lucene.Index.AtomicReader, string)
-        /// 	">Org.Apache.Lucene.Search.FieldCache.GetDocsWithField(Org.Apache.Lucene.Index.AtomicReader, string)
-        /// 	</see>
-        /// .
+        /// <see cref="FieldCache.GetDocsWithField(AtomicReader, string)"/>.
         /// Passing null will assume all docs have spatial data.
         /// </param>
         public DisjointSpatialFilter(SpatialStrategy strategy, SpatialArgs args, string field)
         {
             this.field = field;
+
             // TODO consider making SpatialArgs cloneable
             SpatialOperation origOp = args.Operation; //copy so we can restore
             args.Operation = SpatialOperation.Intersects; //temporarily set to intersects
@@ -109,11 +107,13 @@ namespace Lucene.Net.Spatial
                 // intersects filter against the world bounds. So do we add a method to the
                 // strategy, perhaps?  But the strategy can't cache it.
                 docsWithField = FieldCache.DEFAULT.GetDocsWithField((context.AtomicReader), field);
+
                 int maxDoc = context.AtomicReader.MaxDoc;
                 if (docsWithField.Length() != maxDoc)
                 {
                     throw new InvalidOperationException("Bits length should be maxDoc (" + maxDoc + ") but wasn't: " + docsWithField);
                 }
+
                 if (docsWithField is Bits_MatchNoBits)
                 {
                     return null;//match nothing

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
index 0591a35..666642a 100644
--- a/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
@@ -84,7 +84,7 @@ namespace Lucene.Net.Spatial.Prefix
         /// </summary>
         public abstract class BaseTermsEnumTraverser
         {
-            private readonly AbstractPrefixTreeFilter outerInstance;
+            protected readonly AbstractPrefixTreeFilter outerInstance;
             protected readonly AtomicReaderContext context;
             protected Bits acceptDocs;
             protected readonly int maxDoc;
@@ -92,8 +92,6 @@ namespace Lucene.Net.Spatial.Prefix
             protected TermsEnum termsEnum;//remember to check for null in getDocIdSet
             protected DocsEnum docsEnum;
             
-
-            /// <exception cref="System.IO.IOException"></exception>
             public BaseTermsEnumTraverser(AbstractPrefixTreeFilter outerInstance, AtomicReaderContext context, Bits acceptDocs)
             {
                 this.outerInstance = outerInstance;
@@ -102,14 +100,13 @@ namespace Lucene.Net.Spatial.Prefix
                 AtomicReader reader = context.AtomicReader;
                 this.acceptDocs = acceptDocs;
                 maxDoc = reader.MaxDoc;
-                Terms terms = reader.Terms(this.outerInstance.fieldName);
+                Terms terms = reader.Terms(outerInstance.fieldName);
                 if (terms != null)
                 {
                     termsEnum = terms.Iterator(null);
                 }
             }
 
-            /// <exception cref="System.IO.IOException"></exception>
             protected virtual void CollectDocs(FixedBitSet bitSet)
             {
                 //WARN: keep this specialization in sync
@@ -124,5 +121,22 @@ namespace Lucene.Net.Spatial.Prefix
         }
 
         #endregion
+
+        /* Eventually uncomment when needed.
+
+        protected void collectDocs(Collector collector) throws IOException {
+          //WARN: keep this specialization in sync
+          assert termsEnum != null;
+          docsEnum = termsEnum.docs(acceptDocs, docsEnum, DocsEnum.FLAG_NONE);
+          int docid;
+          while ((docid = docsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+            collector.collect(docid);
+          }
+        }
+
+        public abstract class Collector {
+          abstract void collect(int docid) throws IOException;
+        }
+        */
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
index 3118351..d3aa5df 100644
--- a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
@@ -43,7 +43,7 @@ namespace Lucene.Net.Spatial.Prefix
         // Historical note: this code resulted from a refactoring of RecursivePrefixTreeFilter,
         // which in turn came out of SOLR-2155
 
-        protected internal readonly int prefixGridScanLevel;
+        protected internal readonly int prefixGridScanLevel;//at least one less than grid.getMaxLevels()
 
         public AbstractVisitingPrefixTreeFilter(IShape queryShape, string fieldName, SpatialPrefixTree grid, 
                                                 int detailLevel, int prefixGridScanLevel)
@@ -59,64 +59,18 @@ namespace Lucene.Net.Spatial.Prefix
             {
                 return false;//checks getClass == o.getClass & instanceof
             }
-            
-            var that = (AbstractVisitingPrefixTreeFilter)o;
-            if (prefixGridScanLevel != that.prefixGridScanLevel)
-            {
-                return false;
-            }
+
+            //Ignore prefixGridScanLevel as it is merely a tuning parameter.
+
             return true;
         }
 
         public override int GetHashCode()
         {
             int result = base.GetHashCode();
-            result = 31 * result + prefixGridScanLevel;
             return result;
         }
 
-        #region Nested type: VNode
-
-        /// <summary>
-        /// A Visitor Cell/Cell found via the query shape for
-        /// <see cref="VisitorTemplate">VisitorTemplate</see>
-        /// .
-        /// Sometimes these are reset(cell). It's like a LinkedList node but forms a
-        /// tree.
-        /// </summary>
-        /// <lucene.internal></lucene.internal>
-        public class VNode
-        {
-            internal readonly VNode parent;
-
-            internal Cell cell;
-            internal IEnumerator<VNode> children;
-
-            /// <summary>call reset(cell) after to set the cell.</summary>
-            /// <remarks>call reset(cell) after to set the cell.</remarks>
-            internal VNode(VNode parent)
-            {
-                //Note: The VNode tree adds more code to debug/maintain v.s. a flattened
-                // LinkedList that we used to have. There is more opportunity here for
-                // custom behavior (see preSiblings & postSiblings) but that's not
-                // leveraged yet. Maybe this is slightly more GC friendly.
-                //only null at the root
-                //null, then sometimes set, then null
-                //not null (except initially before reset())
-                // remember to call reset(cell) after
-                this.parent = parent;
-            }
-
-            internal virtual void Reset(Cell cell)
-            {
-                Debug.Assert(cell != null);
-                this.cell = cell;
-                Debug.Assert(children == null);
-            }
-        }
-
-        #endregion
-
         #region Nested type: VisitorTemplate
 
         /// <summary>
@@ -159,7 +113,25 @@ namespace Lucene.Net.Spatial.Prefix
         /// <lucene.internal></lucene.internal>
         public abstract class VisitorTemplate : BaseTermsEnumTraverser
         {
-            private readonly AbstractVisitingPrefixTreeFilter outerInstance;
+            /* Future potential optimizations:
+
+            * Can a polygon query shape be optimized / made-simpler at recursive depths
+              (e.g. intersection of shape + cell box)
+
+            * RE "scan" vs divide & conquer performance decision:
+              We should use termsEnum.docFreq() as an estimate on the number of places at
+              this depth.  It would be nice if termsEnum knew how many terms
+              start with the current term without having to repeatedly next() & test to find out.
+
+            * Perhaps don't do intermediate seek()'s to cells above detailLevel that have Intersects
+              relation because we won't be collecting those docs any way.  However seeking
+              does act as a short-circuit.  So maybe do some percent of the time or when the level
+              is above some threshold.
+
+            * Each shape.relate(otherShape) result could be cached since much of the same relations
+              will be invoked when multiple segments are involved.
+
+            */
 
             protected internal readonly bool hasIndexedLeaves;//if false then we can skip looking for them
 
@@ -173,7 +145,6 @@ namespace Lucene.Net.Spatial.Prefix
                                    bool hasIndexedLeaves)
                 : base(outerInstance, context, acceptDocs)
             {
-                this.outerInstance = outerInstance;
                 this.hasIndexedLeaves = hasIndexedLeaves;
             }
 
@@ -187,9 +158,9 @@ namespace Lucene.Net.Spatial.Prefix
                 //advance
                 if ((thisTerm = termsEnum.Next()) == null)
                 {
-                    return null;
+                    return null;// all done
                 }
-                // all done
+                
                 curVNode = new VNode(null);
                 curVNode.Reset(outerInstance.grid.WorldCell);
 
@@ -225,9 +196,8 @@ namespace Lucene.Net.Spatial.Prefix
                         {
                             if (parentVNode == null)
                             {
-                                goto main_break;
+                                goto main_break;// all done
                             }
-                            // all done
                             if (parentVNode.children.MoveNext())
                             {
                                 //advance next sibling
@@ -261,9 +231,8 @@ namespace Lucene.Net.Spatial.Prefix
                             TermsEnum.SeekStatus seekStatus = termsEnum.SeekCeil(curVNodeTerm);
                             if (seekStatus == TermsEnum.SeekStatus.END)
                             {
-                                break;
+                                break;// all done
                             }
-                            // all done
                             thisTerm = termsEnum.Term();
                             if (seekStatus == TermsEnum.SeekStatus.NOT_FOUND)
                             {
@@ -275,19 +244,17 @@ namespace Lucene.Net.Spatial.Prefix
                         //advance
                         if ((thisTerm = termsEnum.Next()) == null)
                         {
-                            break;
+                            break;// all done
                         }
-                        // all done
                         if (descend)
                         {
                             AddIntersectingChildren();
                         }
                     }
                     ;
-                }
-            main_break:
-                ;
-                //main loop
+                }//main loop
+                main_break: { }
+                
                 return Finish();
             }
 
@@ -319,32 +286,33 @@ namespace Lucene.Net.Spatial.Prefix
                         //advance
                         if ((thisTerm = termsEnum.Next()) == null)
                         {
-                            return;
+                            return;// all done
                         }
                     }
                 }
-                // all done
+                
                 //Decide whether to continue to divide & conquer, or whether it's time to
                 // scan through terms beneath this cell.
                 // Scanning is a performance optimization trade-off.
+
                 //TODO use termsEnum.docFreq() as heuristic
-                bool scan = cell.Level >= outerInstance.prefixGridScanLevel;
-                //simple heuristic
+                bool scan = cell.Level >= ((AbstractVisitingPrefixTreeFilter)outerInstance).prefixGridScanLevel;//simple heuristic
+
                 if (!scan)
                 {
                     //Divide & conquer (ultimately termsEnum.seek())
+
                     IEnumerator<Cell> subCellsIter = FindSubCellsToVisit(cell);
                     if (!subCellsIter.MoveNext())
                     {
-                        //not expected
-                        return;
+                        return;//not expected
                     }
-                    curVNode.children = new VNodeCellIterator
-                        (this, subCellsIter, new VNode(curVNode));
+                    curVNode.children = new VNodeCellIterator(this, subCellsIter, new VNode(curVNode));
                 }
                 else
                 {
                     //Scan (loop of termsEnum.next())
+
                     Scan(outerInstance.detailLevel);
                 }
             }
@@ -402,15 +370,13 @@ namespace Lucene.Net.Spatial.Prefix
             #region Nested type: VNodeCellIterator
 
             /// <summary>
-            /// Used for
-            /// <see cref="VNode.children">VNode.children</see>
-            /// .
+            /// Used for <see cref="VNode.children">VNode.children</see>.
             /// </summary>
             private class VNodeCellIterator : IEnumerator<VNode>
             {
                 private readonly VisitorTemplate outerInstance;
-                internal readonly IEnumerator<Cell> cellIter;
 
+                internal readonly IEnumerator<Cell> cellIter;
                 private readonly VNode vNode;
                 private bool first = true;
 
@@ -486,12 +452,10 @@ namespace Lucene.Net.Spatial.Prefix
             #endregion
 
             /// <summary>Called first to setup things.</summary>
-            /// <remarks>Called first to setup things.</remarks>
             /// <exception cref="System.IO.IOException"></exception>
             protected internal abstract void Start();
 
             /// <summary>Called last to return the result.</summary>
-            /// <remarks>Called last to return the result.</remarks>
             /// <exception cref="System.IO.IOException"></exception>
             protected internal abstract DocIdSet Finish();
 
@@ -526,12 +490,10 @@ namespace Lucene.Net.Spatial.Prefix
             /// <exception cref="System.IO.IOException"></exception>
             protected internal abstract void VisitScanned(Cell cell);
 
-            /// <exception cref="System.IO.IOException"></exception>
             protected internal virtual void PreSiblings(VNode vNode)
             {
             }
 
-            /// <exception cref="System.IO.IOException"></exception>
             protected internal virtual void PostSiblings(VNode vNode)
             {
             }
@@ -539,5 +501,43 @@ namespace Lucene.Net.Spatial.Prefix
         }
 
         #endregion
+
+        #region Nested type: VNode
+
+        /// <summary>
+        /// A Visitor Cell/Cell found via the query shape for
+        /// <see cref="VisitorTemplate">VisitorTemplate</see>
+        /// .
+        /// Sometimes these are reset(cell). It's like a LinkedList node but forms a
+        /// tree.
+        /// </summary>
+        /// <lucene.internal></lucene.internal>
+        public class VNode
+        {
+            //Note: The VNode tree adds more code to debug/maintain v.s. a flattened
+            // LinkedList that we used to have. There is more opportunity here for
+            // custom behavior (see preSiblings & postSiblings) but that's not
+            // leveraged yet. Maybe this is slightly more GC friendly.
+
+            internal readonly VNode parent;//only null at the root
+            internal IEnumerator<VNode> children;//null, then sometimes set, then null
+            internal Cell cell;//not null (except initially before reset())
+
+            /// <summary>call reset(cell) after to set the cell.</summary>
+            internal VNode(VNode parent)
+            {
+                // remember to call reset(cell) after
+                this.parent = parent;
+            }
+
+            internal virtual void Reset(Cell cell)
+            {
+                Debug.Assert(cell != null);
+                this.cell = cell;
+                Debug.Assert(children == null);
+            }
+        }
+
+        #endregion
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
index ca0eb3d..b36604e 100644
--- a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
@@ -38,6 +38,16 @@ namespace Lucene.Net.Spatial.Prefix
     /// <lucene.experimental></lucene.experimental>
     public class ContainsPrefixTreeFilter : AbstractPrefixTreeFilter
     {
+        // Future optimizations:
+        //   Instead of seekExact, use seekCeil with some leap-frogging, like Intersects does.
+
+        /// <summary>
+        /// If the spatial data for a document is comprised of multiple overlapping or adjacent parts,
+        /// it might fail to match a query shape when doing the CONTAINS predicate when the sum of
+        /// those shapes contain the query shape but none do individually. Set this to false to
+        /// increase performance if you don't care about that circumstance (such as if your indexed
+        /// data doesn't even have such conditions).  See LUCENE-5062.
+        /// </summary>
         protected readonly bool multiOverlappingIndexedShapes;
 
         public ContainsPrefixTreeFilter(IShape queryShape, string fieldName, SpatialPrefixTree grid, int detailLevel, bool multiOverlappingIndexedShapes)
@@ -46,7 +56,18 @@ namespace Lucene.Net.Spatial.Prefix
             this.multiOverlappingIndexedShapes = multiOverlappingIndexedShapes;
         }
 
-        /// <exception cref="System.IO.IOException"></exception>
+        public override bool Equals(object o)
+        {
+            if (!base.Equals(o))
+                return false;
+            return multiOverlappingIndexedShapes == ((ContainsPrefixTreeFilter)o).multiOverlappingIndexedShapes;
+        }
+
+        public override int GetHashCode()
+        {
+            return base.GetHashCode() + (multiOverlappingIndexedShapes ? 1 : 0);
+        }
+
         public override DocIdSet GetDocIdSet(AtomicReaderContext context, Bits acceptDocs)
         {
             return new ContainsVisitor(this, context, acceptDocs).Visit(grid.WorldCell, acceptDocs);
@@ -54,23 +75,12 @@ namespace Lucene.Net.Spatial.Prefix
 
         private class ContainsVisitor : BaseTermsEnumTraverser
         {
-            private readonly IShape queryShape;
-            private readonly int detailLevel;
-            private readonly bool multiOverlappingIndexedShapes;
-            private SpatialPrefixTree grid;
-
-            /// <exception cref="System.IO.IOException"></exception>
             public ContainsVisitor(ContainsPrefixTreeFilter outerInstance, AtomicReaderContext context, Bits acceptDocs)
                 : base(outerInstance, context, acceptDocs)
             {
-                this.queryShape = outerInstance.queryShape;
-                this.detailLevel = outerInstance.detailLevel;
-                this.grid = outerInstance.grid;
-                this.multiOverlappingIndexedShapes = outerInstance.multiOverlappingIndexedShapes;
             }
 
             internal BytesRef termBytes = new BytesRef();
-
             internal Cell nextCell;//see getLeafDocs
 
             /// <remarks>This is the primary algorithm; recursive.  Returns null if finds none.</remarks>
@@ -82,17 +92,19 @@ namespace Lucene.Net.Spatial.Prefix
                     //signals all done
                     return null;
                 }
+
+                ContainsPrefixTreeFilter outerInstance = (ContainsPrefixTreeFilter)base.outerInstance;
+
                 //Leaf docs match all query shape
                 SmallDocSet leafDocs = GetLeafDocs(cell, acceptContains);
-                // Get the AND of all child results
+                // Get the AND of all child results (into combinedSubResults)
                 SmallDocSet combinedSubResults = null;
-
                 //   Optimization: use null subCellsFilter when we know cell is within the query shape.
-                IShape subCellsFilter = queryShape;
-                if (cell.Level != 0 && ((cell.GetShapeRel() == null || cell.GetShapeRel() == SpatialRelation.WITHIN)))
+                IShape subCellsFilter = outerInstance.queryShape;
+                if (cell.Level != 0 && ((cell.GetShapeRel() == SpatialRelation.NULL_VALUE || cell.GetShapeRel() == SpatialRelation.WITHIN)))
                 {
                     subCellsFilter = null;
-                    System.Diagnostics.Debug.Assert(cell.GetShape().Relate(queryShape) == SpatialRelation.WITHIN);
+                    Debug.Assert(cell.GetShape().Relate(outerInstance.queryShape) == SpatialRelation.WITHIN);
                 }
                 ICollection<Cell> subCells = cell.GetSubCells(subCellsFilter);
                 foreach (Cell subCell in subCells)
@@ -101,14 +113,14 @@ namespace Lucene.Net.Spatial.Prefix
                     {
                         combinedSubResults = null;
                     }
-                    else if (subCell.Level == detailLevel)
+                    else if (subCell.Level == outerInstance.detailLevel)
                     {
                         combinedSubResults = GetDocs(subCell, acceptContains);
                     }
-                    else if (!multiOverlappingIndexedShapes && 
+                    else if (!outerInstance.multiOverlappingIndexedShapes && 
                         subCell.GetShapeRel() == SpatialRelation.WITHIN)
                     {
-                        combinedSubResults = GetLeafDocs(subCell, acceptContains);
+                        combinedSubResults = GetLeafDocs(subCell, acceptContains); //recursion
                     }
                     else
                     {
@@ -130,15 +142,14 @@ namespace Lucene.Net.Spatial.Prefix
                     {
                         return combinedSubResults;
                     }
-                    return leafDocs.Union(combinedSubResults);
+                    return leafDocs.Union(combinedSubResults);//union is 'or'
                 }
                 return leafDocs;
             }
 
-            /// <exception cref="System.IO.IOException"></exception>
             private bool SeekExact(Cell cell)
             {
-                System.Diagnostics.Debug.Assert(new BytesRef(cell.GetTokenBytes()).CompareTo(termBytes) > 0);
+                Debug.Assert(new BytesRef(cell.GetTokenBytes()).CompareTo(termBytes) > 0);
                 this.termBytes.Bytes = cell.GetTokenBytes();
                 this.termBytes.Length = this.termBytes.Bytes.Length;
                 if (termsEnum == null)
@@ -146,16 +157,14 @@ namespace Lucene.Net.Spatial.Prefix
                 return this.termsEnum.SeekExact(termBytes);
             }
 
-            /// <exception cref="System.IO.IOException"></exception>
             private SmallDocSet GetDocs(Cell cell, Bits acceptContains)
             {
-                System.Diagnostics.Debug.Assert(new BytesRef(cell.GetTokenBytes()).Equals(termBytes));
+                Debug.Assert(new BytesRef(cell.GetTokenBytes()).Equals(termBytes));
                 return this.CollectDocs(acceptContains);
             }
 
             private Cell lastLeaf = null;//just for assertion
 
-            /// <exception cref="System.IO.IOException"></exception>
             private SmallDocSet GetLeafDocs(Cell leafCell, Bits acceptContains)
             {
                 Debug.Assert(new BytesRef(leafCell.GetTokenBytes()).Equals(termBytes));
@@ -167,11 +176,10 @@ namespace Lucene.Net.Spatial.Prefix
                 BytesRef nextTerm = this.termsEnum.Next();
                 if (nextTerm == null)
                 {
-                    termsEnum = null;
-                    //signals all done
+                    termsEnum = null;//signals all done
                     return null;
                 }
-                nextCell = grid.GetCell(nextTerm.Bytes, nextTerm.Offset, nextTerm.Length, this.nextCell);
+                nextCell = outerInstance.grid.GetCell(nextTerm.Bytes, nextTerm.Offset, nextTerm.Length, this.nextCell);
                 if (nextCell.Level == leafCell.Level && nextCell.IsLeaf())
                 {
                     return CollectDocs(acceptContains);
@@ -182,10 +190,10 @@ namespace Lucene.Net.Spatial.Prefix
                 }
             }
 
-            /// <exception cref="System.IO.IOException"></exception>
             private SmallDocSet CollectDocs(Bits acceptContains)
             {
                 SmallDocSet set = null;
+
                 docsEnum = termsEnum.Docs(acceptContains, docsEnum, DocsEnum.FLAG_NONE);
                 int docid;
                 while ((docid = docsEnum.NextDoc()) != DocIdSetIterator.NO_MORE_DOCS)
@@ -213,7 +221,6 @@ namespace Lucene.Net.Spatial.Prefix
         private class SmallDocSet : DocIdSet, Bits
         {
             private readonly SentinelIntSet intSet;
-
             private int maxInt = 0;
 
             public SmallDocSet(int size)
@@ -221,7 +228,7 @@ namespace Lucene.Net.Spatial.Prefix
                 intSet = new SentinelIntSet(size, -1);
             }
 
-            public bool Get(int index)
+            public virtual bool Get(int index)
             {
                 return intSet.Exists(index);
             }
@@ -235,7 +242,8 @@ namespace Lucene.Net.Spatial.Prefix
                 }
             }
 
-            int Bits.Length()
+            /// <summary>Largest docid.</summary>
+            public virtual int Length()
             {
                 return maxInt;
             }
@@ -290,10 +298,9 @@ namespace Lucene.Net.Spatial.Prefix
                 {
                     this.size = size;
                     this.docs = docs;
-                    this.idx = -1;
                 }
 
-                internal int idx;
+                internal int idx = -1;
 
                 public override int DocID()
                 {
@@ -307,7 +314,6 @@ namespace Lucene.Net.Spatial.Prefix
                     }
                 }
 
-                /// <exception cref="System.IO.IOException"></exception>
                 public override int NextDoc()
                 {
                     if (++idx < size)
@@ -317,7 +323,6 @@ namespace Lucene.Net.Spatial.Prefix
                     return NO_MORE_DOCS;
                 }
 
-                /// <exception cref="System.IO.IOException"></exception>
                 public override int Advance(int target)
                 {
                     //for this small set this is likely faster vs. a binary search
@@ -348,7 +353,7 @@ namespace Lucene.Net.Spatial.Prefix
                     }
                     docs[d++] = v;
                 }
-                System.Diagnostics.Debug.Assert(d == intSet.Size());
+                Debug.Assert(d == intSet.Size());
                 int size = d;
                 //sort them
                 Array.Sort(docs, 0, size);

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
index 3d378e0..a150e37 100644
--- a/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
@@ -75,7 +75,6 @@ namespace Lucene.Net.Spatial.Prefix
                 return results;
             }
 
-            /// <exception cref="System.IO.IOException"></exception>
             protected internal override bool Visit(Cell cell)
             {
                 if (cell.GetShapeRel() == SpatialRelation.WITHIN || cell.Level == outerInstance.detailLevel)
@@ -86,13 +85,11 @@ namespace Lucene.Net.Spatial.Prefix
                 return true;
             }
 
-            /// <exception cref="System.IO.IOException"></exception>
             protected internal override void VisitLeaf(Cell cell)
             {
                 CollectDocs(results);
             }
 
-            /// <exception cref="System.IO.IOException"></exception>
             protected internal override void VisitScanned(Cell cell)
             {
                 if (outerInstance.queryShape.Relate(cell.GetShape()).Intersects())

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
index ea36656..a10e717 100644
--- a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
@@ -97,7 +97,7 @@ namespace Lucene.Net.Spatial.Prefix
 
         protected internal readonly bool simplifyIndexedCells;
         protected internal int defaultFieldValuesArrayLen = 2;
-        protected internal double distErrPct = SpatialArgs.DEFAULT_DISTERRPCT;
+        protected internal double distErrPct = SpatialArgs.DEFAULT_DISTERRPCT;// [ 0 TO 0.5 ]
 
         public PrefixTreeStrategy(SpatialPrefixTree grid, string fieldName, bool simplifyIndexedCells)
             : base(grid.SpatialContext, fieldName)
@@ -155,7 +155,7 @@ namespace Lucene.Net.Spatial.Prefix
         public virtual Field[] CreateIndexableFields(IShape shape, double distErr)
         {
             int detailLevel = grid.GetLevelForDistance(distErr);
-            IList<Cell> cells = grid.GetCells(shape, detailLevel, true, simplifyIndexedCells);
+            IList<Cell> cells = grid.GetCells(shape, detailLevel, true, simplifyIndexedCells);//intermediates cells
 
             //TODO is CellTokenStream supposed to be re-used somehow? see Uwe's comments:
             //  http://code.google.com/p/lucene-spatial-playground/issues/detail?id=4
@@ -175,10 +175,7 @@ namespace Lucene.Net.Spatial.Prefix
             FieldType.Freeze();
         }
 
-        /// <summary>Outputs the tokenString of a cell, and if its a leaf, outputs it again with the leaf byte.
-        /// 	</summary>
-        /// <remarks>Outputs the tokenString of a cell, and if its a leaf, outputs it again with the leaf byte.
-        /// 	</remarks>
+        /// <summary>Outputs the tokenString of a cell, and if its a leaf, outputs it again with the leaf byte.</summary>
         internal sealed class CellTokenStream : TokenStream
         {
             private readonly ICharTermAttribute termAtt;
@@ -191,7 +188,7 @@ namespace Lucene.Net.Spatial.Prefix
                 termAtt = AddAttribute<ICharTermAttribute>();
             }
 
-            internal string nextTokenStringNeedingLeaf;
+            internal string nextTokenStringNeedingLeaf = null;
 
             public override bool IncrementToken()
             {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
index 37b4d41..41ae1d0 100644
--- a/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/RecursivePrefixTreeStrategy.cs
@@ -89,8 +89,9 @@ namespace Lucene.Net.Spatial.Prefix
             }
             else if (op == SpatialOperation.IsWithin)
             {
-                return new WithinPrefixTreeFilter(shape, FieldName, grid, detailLevel, prefixGridScanLevel
-                    , -1); //-1 flag is slower but ensures correct results
+                return new WithinPrefixTreeFilter(
+                    shape, FieldName, grid, detailLevel, prefixGridScanLevel, 
+                    -1); //-1 flag is slower but ensures correct results
             }
             else if (op == SpatialOperation.Contains)
             {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
index d4b5d25..8017eb3 100644
--- a/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/TermQueryPrefixTreeStrategy.cs
@@ -44,11 +44,10 @@ namespace Lucene.Net.Spatial.Prefix
     public class TermQueryPrefixTreeStrategy : PrefixTreeStrategy
     {
         public TermQueryPrefixTreeStrategy(SpatialPrefixTree grid, string fieldName)
-            : base(grid, fieldName, false)
+            : base(grid, fieldName, false)//do not simplify indexed cells
         {
         }
 
-        //do not simplify indexed cells
         public override Filter MakeFilter(SpatialArgs args)
         {
             SpatialOperation op = args.Operation;
@@ -58,9 +57,7 @@ namespace Lucene.Net.Spatial.Prefix
             }
             IShape shape = args.Shape;
             int detailLevel = grid.GetLevelForDistance(args.ResolveDistErr(ctx, distErrPct));
-            IList<Cell> cells = grid.GetCells(shape, detailLevel, false, true);
-            //no parents
-            //simplify
+            IList<Cell> cells = grid.GetCells(shape, detailLevel, false /*no parents*/, true /*simplify*/);
             var terms = new BytesRef[cells.Count];
             int i = 0;
             foreach (Cell cell in cells)

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs b/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
index 8af4cd8..5969a08 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
@@ -60,10 +60,6 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// When set via getSubCells(filter), it is the relationship between this cell
         /// and the given shape filter.
         /// </summary>
-        /// <remarks>
-        /// When set via getSubCells(filter), it is the relationship between this cell
-        /// and the given shape filter.
-        /// </remarks>
         protected internal SpatialRelation shapeRel = SpatialRelation.NULL_VALUE;//set in getSubCells(filter), and via setLeaf().
 
         /// <summary>Always false for points.</summary>
@@ -85,7 +81,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             this.token = token;
             if (token.Length > 0 && token[token.Length - 1] == (char)LEAF_BYTE)
             {
-                this.token = token.Substring(0, token.Length - 1);
+                this.token = token.Substring(0, (token.Length - 1) - 0);
                 SetLeaf();
             }
             if (Level == 0)
@@ -107,15 +103,6 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             B_fixLeaf();
         }
 
-        #region IComparable<Cell> Members
-
-        public virtual int CompareTo(Cell o)
-        {
-            return string.CompareOrdinal(TokenString, o.TokenString);
-        }
-
-        #endregion
-
         public virtual void Reset(byte[] bytes, int off, int len)
         {
             Debug.Assert(Level != 0);
@@ -127,23 +114,6 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             B_fixLeaf();
         }
 
-        public virtual void Reset(string token)
-        {
-            Debug.Assert(Level != 0);
-            this.token = token;
-            shapeRel = SpatialRelation.NULL_VALUE;
-
-            //converting string t0 byte[]
-            //bytes = Encoding.UTF8.GetBytes(token);
-            BytesRef utf8Result = new BytesRef(token.Length);
-            UnicodeUtil.UTF16toUTF8(token.ToCharArray(), 0, token.Length, utf8Result);
-            bytes = utf8Result.Bytes;
-
-            b_off = 0;
-            b_len = bytes.Length;
-            B_fixLeaf();
-        }
-
         private void B_fixLeaf()
         {
             //note that non-point shapes always have the maxLevels cell set with setLeaf
@@ -181,9 +151,9 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             leaf = true;
         }
 
-        /*
-         * Note: doesn't contain a trailing leaf byte.
-         */
+        /// <summary>
+        /// Note: doesn't contain a trailing leaf byte.
+        /// </summary>
         public virtual String TokenString
         {
             get
@@ -195,7 +165,6 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         }
 
         /// <summary>Note: doesn't contain a trailing leaf byte.</summary>
-        /// <remarks>Note: doesn't contain a trailing leaf byte.</remarks>
         public virtual byte[] GetTokenBytes()
         {
             if (bytes != null)
@@ -207,11 +176,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             }
             else
             {
-                //converting string t0 byte[]
-                //bytes = Encoding.UTF8.GetBytes(token);
-                BytesRef utf8Result = new BytesRef(token.Length);
-                UnicodeUtil.UTF16toUTF8(token.ToCharArray(), 0, token.Length, utf8Result);
-                bytes = utf8Result.Bytes;
+                bytes = Encoding.UTF8.GetBytes(token);
                 b_off = 0;
                 b_len = bytes.Length;
             }
@@ -323,6 +288,15 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             return GetShape().Center;
         }
 
+        #region IComparable<Cell> Members
+
+        public virtual int CompareTo(Cell o)
+        {
+            return string.CompareOrdinal(TokenString, o.TokenString);
+        }
+
+        #endregion
+
         #region Equality overrides
 
         public override bool Equals(object obj)

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
index 2218ca0..2c775c8 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
@@ -35,27 +35,45 @@ namespace Lucene.Net.Spatial.Prefix.Tree
     /// <lucene.experimental></lucene.experimental>
     public class GeohashPrefixTree : SpatialPrefixTree
     {
+        #region Nested type: Factory
+
+        /// <summary>
+        /// Factory for creating
+        /// <see cref="GeohashPrefixTree">GeohashPrefixTree</see>
+        /// instances with useful defaults
+        /// </summary>
+        public class Factory : SpatialPrefixTreeFactory
+        {
+            protected internal override int GetLevelForDistance(double degrees)
+            {
+                var grid = new GeohashPrefixTree(ctx, GeohashPrefixTree.MaxLevelsPossible);
+                return grid.GetLevelForDistance(degrees);
+            }
+
+            protected internal override SpatialPrefixTree NewSPT()
+            {
+                return new GeohashPrefixTree(ctx, maxLevels.HasValue ? maxLevels.Value : GeohashPrefixTree.MaxLevelsPossible);
+            }
+        }
+
+        #endregion
+
         public GeohashPrefixTree(SpatialContext ctx, int maxLevels)
             : base(ctx, maxLevels)
         {
             IRectangle bounds = ctx.WorldBounds;
             if (bounds.MinX != -180)
             {
-                throw new ArgumentException("Geohash only supports lat-lon world bounds. Got " +
-                                            bounds);
+                throw new ArgumentException("Geohash only supports lat-lon world bounds. Got " + bounds);
             }
             int Maxp = MaxLevelsPossible;
             if (maxLevels <= 0 || maxLevels > Maxp)
             {
-                throw new ArgumentException("maxLen must be [1-" + Maxp + "] but got " + maxLevels
-                    );
+                throw new ArgumentException("maxLen must be [1-" + Maxp + "] but got " + maxLevels);
             }
         }
 
-        /// <summary>Any more than this and there's no point (double lat & lon are the same).
-        /// 	</summary>
-        /// <remarks>Any more than this and there's no point (double lat & lon are the same).
-        /// 	</remarks>
+        /// <summary>Any more than this and there's no point (double lat & lon are the same).</summary>
         public static int MaxLevelsPossible
         {
             get { return GeohashUtils.MAX_PRECISION; }
@@ -65,9 +83,9 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         {
             if (dist == 0)
             {
-                return maxLevels;
+                return maxLevels;//short circuit
             }
-            //short circuit
+            
             int level = GeohashUtils.LookupHashLenForWidthHeight(dist, dist);
             return Math.Max(Math.Min(level, maxLevels), 1);
         }
@@ -88,35 +106,10 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             return new GhCell(this, bytes, offset, len);
         }
 
-        #region Nested type: Factory
-
-        /// <summary>
-        /// Factory for creating
-        /// <see cref="GeohashPrefixTree">GeohashPrefixTree</see>
-        /// instances with useful defaults
-        /// </summary>
-        public class Factory : SpatialPrefixTreeFactory
-        {
-            protected internal override int GetLevelForDistance(double degrees)
-            {
-                var grid = new GeohashPrefixTree(ctx, MaxLevelsPossible);
-                return grid.GetLevelForDistance(degrees);
-            }
-
-            protected internal override SpatialPrefixTree NewSPT()
-            {
-                return new GeohashPrefixTree(ctx, maxLevels.HasValue ? maxLevels.Value : MaxLevelsPossible);
-            }
-        }
-
-        #endregion
-
         #region Nested type: GhCell
 
         internal class GhCell : Cell
         {
-            private IShape shape;
-
             internal GhCell(GeohashPrefixTree outerInstance, string token)
                 : base(outerInstance, token)
             {
@@ -135,8 +128,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
 
             protected internal override ICollection<Cell> GetSubCells()
             {
-                string[] hashes = GeohashUtils.GetSubGeohashes(Geohash);
-                //sorted
+                string[] hashes = GeohashUtils.GetSubGeohashes(Geohash);//sorted
                 IList<Cell> cells = new List<Cell>(hashes.Length);
                 foreach (string hash in hashes)
                 {
@@ -147,17 +139,16 @@ namespace Lucene.Net.Spatial.Prefix.Tree
 
             public override int GetSubCellsSize()
             {
-                return 32;
+                return 32;//8x4
             }
 
-            //8x4
             public override Cell GetSubCell(IPoint p)
             {
-                return outerInstance.GetCell(p, Level + 1);
+                return outerInstance.GetCell(p, Level + 1);//not performant!
             }
 
-            //not performant!
-            //cache
+            private IShape shape;//cache
+
             public override IShape GetShape()
             {
                 if (shape == null)

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
index 26763cc..ce6f48b 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
@@ -34,47 +34,69 @@ namespace Lucene.Net.Spatial.Prefix.Tree
     /// <lucene.experimental></lucene.experimental>
     public class QuadPrefixTree : SpatialPrefixTree
     {
-        public const int MaxLevelsPossible = 50;
+        #region Nested type: Factory
 
-        public const int DefaultMaxLevels = 12;
+        /// <summary>
+        /// Factory for creating
+        /// <see cref="QuadPrefixTree">QuadPrefixTree</see>
+        /// instances with useful defaults
+        /// </summary>
+        public class Factory : SpatialPrefixTreeFactory
+        {
+            protected internal override int GetLevelForDistance(double degrees)
+            {
+                var grid = new QuadPrefixTree(ctx, MaxLevelsPossible);
+                return grid.GetLevelForDistance(degrees);
+            }
 
-        public readonly double gridH;
-        private readonly double gridW;
+            protected internal override SpatialPrefixTree NewSPT()
+            {
+                return new QuadPrefixTree(ctx, maxLevels.HasValue ? maxLevels.Value : MaxLevelsPossible);
+            }
+        }
 
-        internal readonly double[] levelH;
+        #endregion
 
-        internal readonly int[] levelN;
-        internal readonly int[] levelS;
-        internal readonly double[] levelW;
-        private readonly double xmax;
-        private readonly double xmid;
+        public const int MaxLevelsPossible = 50;//not really sure how big this should be
+
+        public const int DefaultMaxLevels = 12;
         private readonly double xmin;
+        private readonly double xmax;
+        private readonly double ymin;
         private readonly double ymax;
+        private readonly double xmid;
         private readonly double ymid;
-        private readonly double ymin;
+
+        private readonly double gridW;
+        public readonly double gridH;
+        
+        internal readonly double[] levelW;
+        internal readonly double[] levelH;
+        internal readonly int[] levelS; // side
+        internal readonly int[] levelN; // number
 
         public QuadPrefixTree(SpatialContext ctx, IRectangle bounds, int maxLevels)
             : base(ctx, maxLevels)
         {
-            //not really sure how big this should be
-            // side
-            // number
             xmin = bounds.MinX;
             xmax = bounds.MaxX;
             ymin = bounds.MinY;
             ymax = bounds.MaxY;
+
             levelW = new double[maxLevels];
             levelH = new double[maxLevels];
             levelS = new int[maxLevels];
             levelN = new int[maxLevels];
+
             gridW = xmax - xmin;
             gridH = ymax - ymin;
-            xmid = xmin + gridW / 2.0;
-            ymid = ymin + gridH / 2.0;
+            this.xmid = xmin + gridW / 2.0;
+            this.ymid = ymin + gridH / 2.0;
             levelW[0] = gridW / 2.0;
             levelH[0] = gridH / 2.0;
             levelS[0] = 2;
             levelN[0] = 4;
+
             for (int i = 1; i < levelW.Length; i++)
             {
                 levelW[i] = levelW[i - 1] / 2.0;
@@ -111,9 +133,8 @@ namespace Lucene.Net.Spatial.Prefix.Tree
 
         public override int GetLevelForDistance(double dist)
         {
-            if (dist == 0)
+            if (dist == 0)//short circuit
             {
-                //short circuit
                 return maxLevels;
             }
             for (int i = 0; i < maxLevels - 1; i++)
@@ -145,12 +166,19 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             return new QuadCell(this, bytes, offset, len);
         }
 
-        private void Build(double x, double y, int level, IList<Cell> matches, StringBuilder
-                                                                                   str, IShape shape, int maxLevel)
+        private void Build(
+            double x, 
+            double y, 
+            int level, 
+            IList<Cell> matches, 
+            StringBuilder str, 
+            IShape shape, 
+            int maxLevel)
         {
             Debug.Assert(str.Length == level);
             double w = levelW[level] / 2;
             double h = levelH[level] / 2;
+
             // Z-Order
             // http://en.wikipedia.org/wiki/Z-order_%28curve%29
             CheckBattenberg('A', x - w, y + h, level, matches, str, shape, maxLevel);
@@ -163,13 +191,20 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         // http://en.wikipedia.org/wiki/Hilbert_curve
         // http://blog.notdot.net/2009/11/Damn-Cool-Algorithms-Spatial-indexing-with-Quadtrees-and-Hilbert-Curves
         // if we actually use the range property in the query, this could be useful
-        private void CheckBattenberg(char c, double cx, double cy, int level, IList<Cell>
-                                                                                  matches, StringBuilder str,
-                                     IShape shape, int maxLevel)
+        private void CheckBattenberg(
+            char c, 
+            double cx, 
+            double cy, 
+            int level, 
+            IList<Cell> matches, 
+            StringBuilder str,
+            IShape shape, 
+            int maxLevel)
         {
             Debug.Assert(str.Length == level);
             double w = levelW[level] / 2;
             double h = levelH[level] / 2;
+
             int strlen = str.Length;
             IRectangle rectangle = ctx.MakeRectangle(cx - w, cx + w, cy - h, cy + h);
             SpatialRelation v = shape.Relate(rectangle);
@@ -179,67 +214,38 @@ namespace Lucene.Net.Spatial.Prefix.Tree
                 //str.append(SpatialPrefixGrid.COVER);
                 matches.Add(new QuadCell(this, str.ToString(), v.Transpose()));
             }
-            else
+            else if (SpatialRelation.DISJOINT == v)
             {
-                if (SpatialRelation.DISJOINT == v)
+                // nothing
+            }
+            else // SpatialRelation.WITHIN, SpatialRelation.INTERSECTS
+            {
+                str.Append(c);
+
+                int nextLevel = level + 1;
+                if (nextLevel >= maxLevel)
                 {
+                    //str.append(SpatialPrefixGrid.INTERSECTS);
+                    matches.Add(new QuadCell(this, str.ToString(), v.Transpose()));
                 }
                 else
                 {
-                    // nothing
-                    // SpatialRelation.WITHIN, SpatialRelation.INTERSECTS
-                    str.Append(c);
-                    int nextLevel = level + 1;
-                    if (nextLevel >= maxLevel)
-                    {
-                        //str.append(SpatialPrefixGrid.INTERSECTS);
-                        matches.Add(new QuadCell(this, str.ToString(), v.Transpose()));
-                    }
-                    else
-                    {
-                        Build(cx, cy, nextLevel, matches, str, shape, maxLevel);
-                    }
+                    Build(cx, cy, nextLevel, matches, str, shape, maxLevel);
                 }
             }
             str.Length = strlen;
         }
 
-        #region Nested type: Factory
-
-        /// <summary>
-        /// Factory for creating
-        /// <see cref="QuadPrefixTree">QuadPrefixTree</see>
-        /// instances with useful defaults
-        /// </summary>
-        public class Factory : SpatialPrefixTreeFactory
-        {
-            protected internal override int GetLevelForDistance(double degrees)
-            {
-                var grid = new QuadPrefixTree(ctx, MaxLevelsPossible);
-                return grid.GetLevelForDistance(degrees);
-            }
-
-            protected internal override SpatialPrefixTree NewSPT()
-            {
-                return new QuadPrefixTree(ctx, maxLevels.HasValue ? maxLevels.Value : MaxLevelsPossible);
-            }
-        }
-
-        #endregion
-
         #region Nested type: QuadCell
 
         internal class QuadCell : Cell
         {
-            private IShape shape;
-
             public QuadCell(QuadPrefixTree outerInstance, string token)
                 : base(outerInstance, token)
             {
             }
 
-            public QuadCell(QuadPrefixTree outerInstance, string token, SpatialRelation shapeRel
-                )
+            public QuadCell(QuadPrefixTree outerInstance, string token, SpatialRelation shapeRel)
                 : base(outerInstance, token)
             {
                 this.shapeRel = shapeRel;
@@ -274,11 +280,11 @@ namespace Lucene.Net.Spatial.Prefix.Tree
 
             public override Cell GetSubCell(IPoint p)
             {
-                return outerInstance.GetCell(p, Level + 1);
+                return outerInstance.GetCell(p, Level + 1);//not performant!
             }
 
-            //not performant!
-            //cache
+            private IShape shape; //cache
+
             public override IShape GetShape()
             {
                 if (shape == null)
@@ -301,31 +307,22 @@ namespace Lucene.Net.Spatial.Prefix.Tree
                     {
                         ymin += outerInstance.levelH[i];
                     }
+                    else if ('B' == c || 'b' == c)
+                    {
+                        xmin += outerInstance.levelW[i];
+                        ymin += outerInstance.levelH[i];
+                    }
+                    else if ('C' == c || 'c' == c)
+                    {
+                        // nothing really
+                    }
+                    else if ('D' == c || 'd' == c)
+                    {
+                        xmin += outerInstance.levelW[i];
+                    }
                     else
                     {
-                        if ('B' == c || 'b' == c)
-                        {
-                            xmin += outerInstance.levelW[i];
-                            ymin += outerInstance.levelH[i];
-                        }
-                        else
-                        {
-                            if ('C' == c || 'c' == c)
-                            {
-                            }
-                            else
-                            {
-                                // nothing really
-                                if ('D' == c || 'd' == c)
-                                {
-                                    xmin += outerInstance.levelW[i];
-                                }
-                                else
-                                {
-                                    throw new Exception("unexpected char: " + c);
-                                }
-                            }
-                        }
+                        throw new Exception("unexpected char: " + c);
                     }
                 }
                 int len = token.Length;
@@ -343,9 +340,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
                 }
                 return outerInstance.ctx.MakeRectangle(xmin, xmin + width, ymin, ymin + height);
             }
-
-            //QuadCell
-        }
+        }//QuadCell
 
         #endregion
     }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
index 77f61fb..9964485 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
@@ -111,10 +111,9 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             return Math.Sqrt(width * width + height * height);
         }
 
-        [System.NonSerialized]
-        private Cell worldCell;
+        [NonSerialized]
+        private Cell worldCell;//cached
 
-        //cached
         /// <summary>Returns the level 0 cell which encompasses all spatial data.</summary>
         /// <remarks>
         /// Returns the level 0 cell which encompasses all spatial data. Equivalent to
@@ -223,9 +222,8 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         {
             if (cell.Level == detailLevel)
             {
-                cell.SetLeaf();
+                cell.SetLeaf();//FYI might already be a leaf
             }
-            //FYI might already be a leaf
             if (cell.IsLeaf())
             {
                 result.Add(cell);
@@ -235,6 +233,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             {
                 result.Add(cell);
             }
+
             ICollection<Cell> subCells = cell.GetSubCells(shape);
             int leaves = 0;
             foreach (Cell subCell in subCells)
@@ -247,20 +246,19 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             //can we simplify?
             if (simplify && leaves == cell.GetSubCellsSize() && cell.Level != 0)
             {
+                //Optimization: substitute the parent as a leaf instead of adding all
+                // children as leaves
+
+                //remove the leaves
                 do
                 {
-                    //Optimization: substitute the parent as a leaf instead of adding all
-                    // children as leaves
-                    //remove the leaves
-                    result.RemoveAt(result.Count - 1);
+                    result.RemoveAt(result.Count - 1);//remove last
                 }
                 while (--leaves > 0);
-                //remove last
                 //add cell as the leaf
                 cell.SetLeaf();
-                if (!inclParents)
+                if (!inclParents)// otherwise it was already added up above
                 {
-                    // otherwise it was already added up above
                     result.Add(cell);
                 }
                 return true;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
index cca47ec..ad70563 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
@@ -31,21 +31,15 @@ namespace Lucene.Net.Spatial.Prefix.Tree
     /// <lucene.experimental></lucene.experimental>
     public abstract class SpatialPrefixTreeFactory
     {
-        private const double DefaultGeoMaxDetailKm = 0.001;
-
+        private const double DefaultGeoMaxDetailKm = 0.001;//1m
         public const string PrefixTree = "prefixTree";
-
         public const string MaxLevels = "maxLevels";
-
         public const string MaxDistErr = "maxDistErr";
 
         protected internal IDictionary<string, string> args;
-
         protected internal SpatialContext ctx;
-
         protected internal int? maxLevels;
 
-        //1m
         /// <summary>The factory  is looked up via "prefixTree" in args, expecting "geohash" or "quad".
         /// 	</summary>
         /// <remarks>
@@ -55,8 +49,8 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         public static SpatialPrefixTree MakeSPT(IDictionary<string, string> args, SpatialContext ctx)
         {
             SpatialPrefixTreeFactory instance;
-            string cname = args[PrefixTree];
-            if (cname == null)
+            string cname;
+            if (!args.TryGetValue(PrefixTree, out cname))
             {
                 cname = ctx.IsGeo ? "geohash" : "quad";
             }
@@ -64,23 +58,20 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             {
                 instance = new GeohashPrefixTree.Factory();
             }
+            else if ("quad".Equals(cname, StringComparison.OrdinalIgnoreCase))
+            {
+                instance = new QuadPrefixTree.Factory();
+            }
             else
             {
-                if ("quad".Equals(cname, StringComparison.OrdinalIgnoreCase))
+                try
                 {
-                    instance = new QuadPrefixTree.Factory();
+                    Type c = Type.GetType(cname);
+                    instance = (SpatialPrefixTreeFactory)Activator.CreateInstance(c);
                 }
-                else
+                catch (Exception e)
                 {
-                    try
-                    {
-                        Type c = Type.GetType(cname);
-                        instance = (SpatialPrefixTreeFactory)Activator.CreateInstance(c);
-                    }
-                    catch (Exception e)
-                    {
-                        throw new Exception(string.Empty, e);
-                    }
+                    throw new ApplicationException(string.Empty, e);
                 }
             }
             instance.Init(args, ctx);
@@ -96,15 +87,15 @@ namespace Lucene.Net.Spatial.Prefix.Tree
 
         protected internal virtual void InitMaxLevels()
         {
-            string mlStr = args[MaxLevels];
-            if (mlStr != null)
+            string mlStr;
+            if (args.TryGetValue(MaxLevels, out mlStr))
             {
                 maxLevels = int.Parse(mlStr, CultureInfo.InvariantCulture);
                 return;
             }
             double degrees;
-            string maxDetailDistStr = args[MaxDistErr];
-            if (maxDetailDistStr == null)
+            string maxDetailDistStr;
+            if (!args.TryGetValue(MaxDistErr, out maxDetailDistStr))
             {
                 if (!ctx.IsGeo)
                 {
@@ -115,16 +106,14 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             }
             else
             {
-                degrees = double.Parse(maxDetailDistStr);
+                degrees = double.Parse(maxDetailDistStr, CultureInfo.InvariantCulture);
             }
             maxLevels = GetLevelForDistance(degrees);
         }
 
         /// <summary>
         /// Calls
-        /// <see cref="SpatialPrefixTree.GetLevelForDistance(double)">SpatialPrefixTree.GetLevelForDistance(double)
-        /// 	</see>
-        /// .
+        /// <see cref="SpatialPrefixTree.GetLevelForDistance(double)">SpatialPrefixTree.GetLevelForDistance(double)</see>.
         /// </summary>
         protected internal abstract int GetLevelForDistance(double degrees);
 

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
index 11cfc46..110739f 100644
--- a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
@@ -93,56 +93,53 @@ namespace Lucene.Net.Spatial.Prefix
             {
                 return ctx.MakeCircle((IPoint)shape, distErr);
             }
+            else if (shape is ICircle)
+            {
+                var circle = (ICircle)shape;
+                double newDist = circle.Radius + distErr;
+                if (ctx.IsGeo && newDist > 180)
+                {
+                    newDist = 180;
+                }
+                return ctx.MakeCircle(circle.Center, newDist);
+            }
             else
             {
-                if (shape is ICircle)
+                IRectangle bbox = shape.BoundingBox;
+                double newMinX = bbox.MinX - distErr;
+                double newMaxX = bbox.MaxX + distErr;
+                double newMinY = bbox.MinY - distErr;
+                double newMaxY = bbox.MaxY + distErr;
+                if (ctx.IsGeo)
                 {
-                    var circle = (ICircle)shape;
-                    double newDist = circle.Radius + distErr;
-                    if (ctx.IsGeo && newDist > 180)
+                    if (newMinY < -90)
                     {
-                        newDist = 180;
+                        newMinY = -90;
                     }
-                    return ctx.MakeCircle(circle.Center, newDist);
-                }
-                else
-                {
-                    IRectangle bbox = shape.BoundingBox;
-                    double newMinX = bbox.MinX - distErr;
-                    double newMaxX = bbox.MaxX + distErr;
-                    double newMinY = bbox.MinY - distErr;
-                    double newMaxY = bbox.MaxY + distErr;
-                    if (ctx.IsGeo)
+                    if (newMaxY > 90)
+                    {
+                        newMaxY = 90;
+                    }
+                    if (newMinY == -90 || newMaxY == 90 || bbox.Width + 2 * distErr > 360)
                     {
-                        if (newMinY < -90)
-                        {
-                            newMinY = -90;
-                        }
-                        if (newMaxY > 90)
-                        {
-                            newMaxY = 90;
-                        }
-                        if (newMinY == -90 || newMaxY == 90 || bbox.Width + 2 * distErr > 360)
-                        {
-                            newMinX = -180;
-                            newMaxX = 180;
-                        }
-                        else
-                        {
-                            newMinX = DistanceUtils.NormLonDEG(newMinX);
-                            newMaxX = DistanceUtils.NormLonDEG(newMaxX);
-                        }
+                        newMinX = -180;
+                        newMaxX = 180;
                     }
                     else
                     {
-                        //restrict to world bounds
-                        newMinX = Math.Max(newMinX, ctx.WorldBounds.MinX);
-                        newMaxX = Math.Min(newMaxX, ctx.WorldBounds.MaxX);
-                        newMinY = Math.Max(newMinY, ctx.WorldBounds.MinY);
-                        newMaxY = Math.Min(newMaxY, ctx.WorldBounds.MaxY);
+                        newMinX = DistanceUtils.NormLonDEG(newMinX);
+                        newMaxX = DistanceUtils.NormLonDEG(newMaxX);
                     }
-                    return ctx.MakeRectangle(newMinX, newMaxX, newMinY, newMaxY);
                 }
+                else
+                {
+                    //restrict to world bounds
+                    newMinX = Math.Max(newMinX, ctx.WorldBounds.MinX);
+                    newMaxX = Math.Min(newMaxX, ctx.WorldBounds.MaxX);
+                    newMinY = Math.Max(newMinY, ctx.WorldBounds.MinY);
+                    newMaxY = Math.Min(newMaxY, ctx.WorldBounds.MaxY);
+                }
+                return ctx.MakeRectangle(newMinX, newMaxX, newMinY, newMaxY);
             }
         }
 
@@ -157,10 +154,9 @@ namespace Lucene.Net.Spatial.Prefix
         private sealed class _VisitorTemplate_121 : VisitorTemplate
         {
             private readonly WithinPrefixTreeFilter outerInstance;
-            private FixedBitSet inside;
 
+            private FixedBitSet inside;
             private FixedBitSet outside;
-
             private SpatialRelation visitRelation;
 
             public _VisitorTemplate_121(WithinPrefixTreeFilter outerInstance, AtomicReaderContext context, 
@@ -188,7 +184,6 @@ namespace Lucene.Net.Spatial.Prefix
                 return cell.GetSubCells(outerInstance.bufferedQueryShape).GetEnumerator();
             }
 
-            /// <exception cref="System.IO.IOException"></exception>
             protected internal override bool Visit(Cell cell)
             {
                 //cell.relate is based on the bufferedQueryShape; we need to examine what
@@ -236,7 +231,7 @@ namespace Lucene.Net.Spatial.Prefix
             /// Returns true if the provided cell, and all its sub-cells down to
             /// detailLevel all intersect the queryShape.
             /// </remarks>
-            private bool AllCellsIntersectQuery(Cell cell, SpatialRelation relate)
+            private bool AllCellsIntersectQuery(Cell cell, SpatialRelation relate/*cell to query*/)
             {
                 if (relate == SpatialRelation.NULL_VALUE)
                 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialArgs.cs b/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
index 0ac68d2..83acc37 100644
--- a/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
+++ b/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
@@ -25,7 +25,10 @@ namespace Lucene.Net.Spatial.Queries
     {
         public static readonly double DEFAULT_DISTERRPCT = 0.025d;
 
-        public SpatialOperation Operation { get; set; }
+        private SpatialOperation operation;
+        private IShape shape;
+        private double? distErrPct;
+        private double? distErr;
 
         public SpatialArgs(SpatialOperation operation, IShape shape)
         {
@@ -67,13 +70,13 @@ namespace Lucene.Net.Spatial.Queries
 
         /// <summary>
         /// Gets the error distance that specifies how precise the query shape is. This
-        /// looks at {@link #getDistErr()}, {@link #getDistErrPct()}, and {@code
-        /// defaultDistErrPct}.
+        /// looks at <see cref="DistErr"/>, <see cref="DistErrPct"/>, and 
+        /// <paramref name="defaultDistErrPct"/>.
         /// </summary>
         /// <param name="ctx"></param>
         /// <param name="defaultDistErrPct">0 to 0.5</param>
         /// <returns>>= 0</returns>
-        public double ResolveDistErr(SpatialContext ctx, double defaultDistErrPct)
+        public virtual double ResolveDistErr(SpatialContext ctx, double defaultDistErrPct)
         {
             if (DistErr != null)
                 return DistErr.Value;
@@ -86,7 +89,7 @@ namespace Lucene.Net.Spatial.Queries
         /// </summary>
         public void Validate()
         {
-            if (Operation.IsTargetNeedsArea() && !Shape.HasArea)
+            if (Operation.IsTargetNeedsArea && !Shape.HasArea)
             {
                 throw new ArgumentException(Operation + " only supports geometry with area");
             }
@@ -97,7 +100,7 @@ namespace Lucene.Net.Spatial.Queries
             }
         }
 
-        public override String ToString()
+        public override string ToString()
         {
             return SpatialArgsParser.WriteSpatialArgs(this);
         }
@@ -106,7 +109,17 @@ namespace Lucene.Net.Spatial.Queries
         // Getters & Setters
         //------------------------------------------------
 
-        public IShape Shape { get; set; }
+        public virtual SpatialOperation Operation
+        {
+            get { return operation; }
+            set { operation = value; }
+        }
+
+        public virtual IShape Shape
+        {
+            get { return shape; }
+            set { shape = value; }
+        }
 
         /// <summary>
         /// A measure of acceptable error of the shape as a fraction. This effectively
@@ -115,22 +128,27 @@ namespace Lucene.Net.Spatial.Queries
         /// The default is {@link #DEFAULT_DIST_PRECISION}
         /// </summary>
         /// <returns>0 to 0.5</returns>
-        public double? DistErrPct
+        public virtual double? DistErrPct
         {
             get { return distErrPct; }
             set
             {
                 if (value != null)
-                    distErrPct = value.Value;
+                {
+                    distErrPct = value;
+                }
             }
         }
-        private double? distErrPct;
 
         /// <summary>
         /// The acceptable error of the shape.  This effectively inflates the
         /// size of the shape but should not shrink it.
         /// </summary>
         /// <returns>>= 0</returns>
-        public double? DistErr { get; set; }
+        public virtual double? DistErr
+        {
+            get { return distErr; }
+            set { distErr = value; }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs b/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
index 8ffad19..e71c9b5 100644
--- a/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
+++ b/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
@@ -20,27 +20,44 @@ using System.Collections.Generic;
 using System.Text;
 using Lucene.Net.Support;
 using Spatial4n.Core.Context;
+using Spatial4n.Core.Exceptions;
+using System.Linq;
+using Spatial4n.Core.Shapes;
 
 namespace Lucene.Net.Spatial.Queries
 {
+    /// <summary>
+    /// Parses a string that usually looks like "OPERATION(SHAPE)" into a {@link SpatialArgs}
+    /// object. The set of operations supported are defined in {@link SpatialOperation}, such
+    /// as "Intersects" being a common one. The shape portion is defined by WKT {@link com.spatial4j.core.io.WktShapeParser},
+    /// but it can be overridden/customized via {@link #parseShape(String, com.spatial4j.core.context.SpatialContext)}.
+    /// There are some optional name-value pair parameters that follow the closing parenthesis.  Example:
+    /// <code>
+    ///   Intersects(ENVELOPE(-10,-8,22,20)) distErrPct=0.025
+    /// </code>
+    /// <para/>
+    /// In the future it would be good to support something at least semi-standardized like a
+    /// variant of <a href="http://docs.geoserver.org/latest/en/user/filter/ecql_reference.html#spatial-predicate">
+    ///   [E]CQL</a>.
+    ///   
+    /// @lucene.experimental
+    /// </summary>
     public class SpatialArgsParser
     {
-        public const String DIST_ERR_PCT = "distErrPct";
-        public const String DIST_ERR = "distErr";
+        public const string DIST_ERR_PCT = "distErrPct";
+        public const string DIST_ERR = "distErr";
 
         /// <summary>
         /// Writes a close approximation to the parsed input format.
         /// </summary>
-        /// <param name="args"></param>
-        /// <returns></returns>
-        public static String WriteSpatialArgs(SpatialArgs args)
+        public static string WriteSpatialArgs(SpatialArgs args)
         {
             var str = new StringBuilder();
             str.Append(args.Operation.Name);
             str.Append('(');
             str.Append(args.Shape);
             if (args.DistErrPct != null)
-                str.Append(" distErrPct=").Append(String.Format("{0:0.00}%", args.DistErrPct * 100d));
+                str.Append(" distErrPct=").Append(string.Format("{0:0.00}%", args.DistErrPct * 100d));
             if (args.DistErr != null)
                 str.Append(" distErr=").Append(args.DistErr);
             str.Append(')');
@@ -50,45 +67,43 @@ namespace Lucene.Net.Spatial.Queries
         /// <summary>
         /// Parses a string such as "Intersects(-10,20,-8,22) distErrPct=0.025".
         /// </summary>
-        /// <param name="v"></param>
-        /// <param name="ctx"></param>
-        /// <returns></returns>
-        public SpatialArgs Parse(String v, SpatialContext ctx)
+        /// <param name="v">The string to parse. Mandatory.</param>
+        /// <param name="ctx">The spatial context. Mandatory.</param>
+        /// <returns>Not null.</returns>
+        /// <exception cref="ArgumentException">if the parameters don't make sense or an add-on parameter is unknown</exception>
+        /// <exception cref="ParseException">If there is a problem parsing the string</exception>
+        /// <exception cref="InvalidShapeException">When the coordinates are invalid for the shape</exception>
+        public SpatialArgs Parse(string v, SpatialContext ctx)
         {
             int idx = v.IndexOf('(');
             int edx = v.LastIndexOf(')');
 
             if (idx < 0 || idx > edx)
             {
-                throw new ArgumentException("missing parens: " + v);
+                throw new ParseException("missing parens: " + v, -1);
             }
 
-            SpatialOperation op = SpatialOperation.Get(v.Substring(0, idx).Trim());
+            SpatialOperation op = SpatialOperation.Get(v.Substring(0, idx - 0).Trim());
 
             //Substring in .NET is (startPosn, length), But in Java it's (startPosn, endPosn)
             //see http://docs.oracle.com/javase/1.4.2/docs/api/java/lang/String.html#substring(int, int)
-            String body = v.Substring(idx + 1, edx - (idx + 1)).Trim();
+            string body = v.Substring(idx + 1, edx - (idx + 1)).Trim();
             if (body.Length < 1)
             {
-                throw new ArgumentException("missing body : " + v);
+                throw new ParseException("missing body : " + v, idx + 1);
             }
 
-            var shape = ctx.ReadShape(body);
-            var args = new SpatialArgs(op, shape);
+            var shape = ParseShape(body, ctx);
+            var args = NewSpatialArgs(op, shape);
 
             if (v.Length > (edx + 1))
             {
                 body = v.Substring(edx + 1).Trim();
                 if (body.Length > 0)
                 {
-                    Dictionary<String, String> aa = ParseMap(body);
-                    args.DistErrPct = ReadDouble(aa[DIST_ERR_PCT]);
-                    aa.Remove(DIST_ERR_PCT);
-
-                    args.DistErr = ReadDouble(aa[DIST_ERR]);
-                    aa.Remove(DIST_ERR);
-
-                    if (aa.Count != 0)
+                    IDictionary<string, string> aa = ParseMap(body);
+                    ReadNameValuePairs(args, aa);
+                    if (!aa.Any())
                     {
                         throw new ArgumentException("unused parameters: " + aa);
                     }
@@ -98,13 +113,35 @@ namespace Lucene.Net.Spatial.Queries
             return args;
         }
 
-        protected static double? ReadDouble(String v)
+        protected virtual SpatialArgs NewSpatialArgs(SpatialOperation op, IShape shape)
+        {
+            return new SpatialArgs(op, shape);
+        }
+
+        protected virtual void ReadNameValuePairs(SpatialArgs args, IDictionary<string, string> nameValPairs)
+        {
+            string distErrPctStr, distErrStr;
+            nameValPairs.TryGetValue(DIST_ERR_PCT, out distErrPctStr);
+            nameValPairs.TryGetValue(DIST_ERR, out distErrStr);
+            args.DistErrPct = ReadDouble(distErrPctStr);
+            nameValPairs.Remove(DIST_ERR_PCT);
+            args.DistErr = ReadDouble(distErrStr);
+            nameValPairs.Remove(DIST_ERR);
+        }
+
+        protected virtual IShape ParseShape(string str, SpatialContext ctx) 
+        {
+            //return ctx.readShape(str);//still in Spatial4n 0.4 but will be deleted
+            return ctx.ReadShapeFromWkt(str);
+        }
+
+        protected static double? ReadDouble(string v)
         {
             double val;
             return double.TryParse(v, out val) ? val : (double?)null;
         }
 
-        protected static bool ReadBool(String v, bool defaultValue)
+        protected static bool ReadBool(string v, bool defaultValue)
         {
             bool ret;
             return bool.TryParse(v, out ret) ? ret : defaultValue;
@@ -114,21 +151,19 @@ namespace Lucene.Net.Spatial.Queries
         /// Parses "a=b c=d f" (whitespace separated) into name-value pairs. If there
         /// is no '=' as in 'f' above then it's short for f=f.
         /// </summary>
-        /// <param name="body"></param>
-        /// <returns></returns>
-        protected static Dictionary<String, String> ParseMap(String body)
+        protected static IDictionary<string, string> ParseMap(string body)
         {
-            var map = new Dictionary<String, String>();
+            var map = new Dictionary<string, string>();
             StringTokenizer st = new StringTokenizer(body, " \n\t");
 
             while (st.HasMoreTokens())
             {
-                String a = st.NextToken();
+                string a = st.NextToken();
                 int idx = a.IndexOf('=');
                 if (idx > 0)
                 {
-                    String k = a.Substring(0, idx);
-                    String v = a.Substring(idx + 1);
+                    string k = a.Substring(0, idx - 0);
+                    string v = a.Substring(idx + 1);
                     map[k] = v;
                 }
                 else


[16/26] lucenenet git commit: Added missing Spatial classes, ported Spatial tests, and fixed several bugs

Posted by ni...@apache.org.
Added missing Spatial classes, ported Spatial tests, and fixed several bugs


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

Branch: refs/heads/master
Commit: d8c73530744a7b540258169f70cd893871e65ffb
Parents: bd9e92d
Author: Shad Storhaug <sh...@shadstorhaug.com>
Authored: Wed Nov 16 14:42:10 2016 +0700
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Wed Nov 16 14:42:10 2016 +0700

----------------------------------------------------------------------
 Lucene.Net.sln                                  |    26 +
 src/Lucene.Net.Spatial/DisjointSpatialFilter.cs |     2 +-
 .../Lucene.Net.Spatial.csproj                   |     2 +
 .../Prefix/AbstractVisitingPrefixTreeFilter.cs  |   196 +-
 .../Prefix/ContainsPrefixTreeFilter.cs          |     5 +-
 .../Prefix/PrefixTreeStrategy.cs                |     4 +-
 src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs      |    44 +-
 .../Prefix/Tree/GeohashPrefixTree.cs            |    19 +-
 .../Prefix/Tree/QuadPrefixTree.cs               |    50 +-
 .../Prefix/Tree/SpatialPrefixTree.cs            |    16 +-
 .../Prefix/Tree/SpatialPrefixTreeFactory.cs     |     8 +-
 .../Serialized/SerializedDVStrategy.cs          |   409 +-
 .../Util/DistanceToShapeValueSource.cs          |   151 +
 .../Util/ShapePredicateValueSource.cs           |   146 +
 .../Vector/PointVectorStrategy.cs               |     2 +-
 .../JavaCompatibility/LuceneTestCase.cs         |   104 +
 .../JavaCompatibility/SystemTypesHelpers.cs     |    27 +-
 .../DistanceStrategyTest.cs                     |   155 +
 .../Lucene.Net.Tests.Spatial.csproj             |   130 +
 src/Lucene.Net.Tests.Spatial/PortedSolr3Test.cs |   201 +
 .../Prefix/NtsPolygonTest.cs                    |   118 +
 .../Prefix/SpatialOpRecursivePrefixTreeTest.cs  |   485 +
 .../Prefix/TestRecursivePrefixTreeStrategy.cs   |   127 +
 .../Prefix/TestTermQueryPrefixGridStrategy.cs   |    60 +
 .../Prefix/Tree/SpatialPrefixTreeTest.cs        |   105 +
 .../Properties/AssemblyInfo.cs                  |    36 +
 .../Query/SpatialArgsParserTest.cs              |    69 +
 .../QueryEqualsHashCodeTest.cs                  |   169 +
 .../Serialized/SerializedStrategyTest.cs        |    73 +
 src/Lucene.Net.Tests.Spatial/SpatialArgsTest.cs |    52 +
 src/Lucene.Net.Tests.Spatial/SpatialExample.cs  |   196 +
 .../SpatialMatchConcern.cs                      |    35 +
 src/Lucene.Net.Tests.Spatial/SpatialTestCase.cs |   248 +
 src/Lucene.Net.Tests.Spatial/SpatialTestData.cs |    78 +
 .../SpatialTestQuery.cs                         |   102 +
 .../StrategyTestCase.cs                         |   274 +
 .../Test-Files/Data/LUCENE-4464.txt             |     3 +
 .../Test-Files/Data/countries-bbox.txt          |   249 +
 .../Test-Files/Data/countries-poly.txt          |   249 +
 .../Test-Files/Data/geonames-IE.txt             | 22929 +++++++++++++++++
 .../Test-Files/Data/simple-bbox.txt             |     5 +
 .../Test-Files/Data/states-bbox.txt             |    52 +
 .../Test-Files/Data/states-poly.txt             |    52 +
 .../Test-Files/Data/world-cities-points.txt     |  2680 ++
 .../Test-Files/cities-Intersects-BBox.txt       |     3 +
 .../Test-Files/simple-Queries-BBox.txt          |     8 +
 .../Test-Files/states-Intersects-BBox.txt       |     3 +
 .../Test-Files/states-IsWithin-BBox.txt         |     4 +
 .../TestTestFramework.cs                        |    63 +
 .../Vector/TestPointVectorStrategy.cs           |    61 +
 src/Lucene.Net.Tests.Spatial/packages.config    |     4 +
 51 files changed, 30132 insertions(+), 157 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/Lucene.Net.sln
----------------------------------------------------------------------
diff --git a/Lucene.Net.sln b/Lucene.Net.sln
index 136a3fe..e7a3ef8 100644
--- a/Lucene.Net.sln
+++ b/Lucene.Net.sln
@@ -72,6 +72,8 @@ Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Lucene.Net.Tests.Grouping",
 EndProject
 Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Lucene.Net.Spatial", "src\Lucene.Net.Spatial\Lucene.Net.Spatial.csproj", "{35C347F4-24B2-4BE5-8117-A0E3001551CE}"
 EndProject
+Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Lucene.Net.Tests.Spatial", "src\Lucene.Net.Tests.Spatial\Lucene.Net.Tests.Spatial.csproj", "{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}"
+EndProject
 Global
 	GlobalSection(SolutionConfigurationPlatforms) = preSolution
 		Debug|Any CPU = Debug|Any CPU
@@ -711,6 +713,30 @@ Global
 		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
 		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Release35|x86.ActiveCfg = Release|Any CPU
 		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Release35|x86.Build.0 = Release|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Debug|Any CPU.Build.0 = Debug|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Debug|x86.ActiveCfg = Debug|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Debug|x86.Build.0 = Debug|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Debug35|Any CPU.Build.0 = Debug|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Debug35|Mixed Platforms.Build.0 = Debug|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Debug35|x86.ActiveCfg = Debug|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Debug35|x86.Build.0 = Debug|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Release|Any CPU.ActiveCfg = Release|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Release|Any CPU.Build.0 = Release|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Release|Mixed Platforms.Build.0 = Release|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Release|x86.ActiveCfg = Release|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Release|x86.Build.0 = Release|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Release35|Any CPU.Build.0 = Release|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Release35|x86.ActiveCfg = Release|Any CPU
+		{31F52F5C-A08F-4363-8003-23D6F7D6EB3A}.Release35|x86.Build.0 = Release|Any CPU
 	EndGlobalSection
 	GlobalSection(SolutionProperties) = preSolution
 		HideSolutionNode = FALSE

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs b/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
index e36a97b..41842af 100644
--- a/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
+++ b/src/Lucene.Net.Spatial/DisjointSpatialFilter.cs
@@ -125,7 +125,7 @@ namespace Lucene.Net.Spatial
             }
             
             //not so much a chain but a way to conveniently invert the Filter
-            DocIdSet docIdSet = new ChainedFilter(new[] { intersectsFilter }, ChainedFilter.ANDNOT).GetDocIdSet(context, acceptDocs);
+            DocIdSet docIdSet = new ChainedFilter(new Filter[] { intersectsFilter }, ChainedFilter.ANDNOT).GetDocIdSet(context, acceptDocs);
             return BitsFilteredDocIdSet.Wrap(docIdSet, docsWithField);
         }
     }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj b/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj
index 2689008..28c3e51 100644
--- a/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj
+++ b/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj
@@ -100,10 +100,12 @@
     <Compile Include="Query\UnsupportedSpatialOperation.cs" />
     <Compile Include="Serialized\SerializedDVStrategy.cs" />
     <Compile Include="SpatialStrategy.cs" />
+    <Compile Include="Util\DistanceToShapeValueSource.cs" />
     <Compile Include="Util\ShapeFieldCacheDistanceValueSource.cs" />
     <Compile Include="Util\CachingDoubleValueSource.cs" />
     <Compile Include="Util\ShapeFieldCache.cs" />
     <Compile Include="Util\ShapeFieldCacheProvider.cs" />
+    <Compile Include="Util\ShapePredicateValueSource.cs" />
     <Compile Include="Util\ValueSourceFilter.cs" />
     <Compile Include="Vector\DistanceValueSource.cs" />
     <Compile Include="Vector\PointVectorStrategy.cs" />

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
index e368a55..3118351 100644
--- a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
@@ -160,14 +160,15 @@ namespace Lucene.Net.Spatial.Prefix
         public abstract class VisitorTemplate : BaseTermsEnumTraverser
         {
             private readonly AbstractVisitingPrefixTreeFilter outerInstance;
-            private readonly BytesRef curVNodeTerm = new BytesRef();
+
             protected internal readonly bool hasIndexedLeaves;//if false then we can skip looking for them
 
             private VNode curVNode;//current pointer, derived from query shape
+            private BytesRef curVNodeTerm = new BytesRef();//curVNode.cell's term.
+            private Cell scanCell;
+
             private BytesRef thisTerm; //the result of termsEnum.term()
-            private Cell scanCell;//curVNode.cell's term.
 
-            /// <exception cref="System.IO.IOException"></exception>
             public VisitorTemplate(AbstractVisitingPrefixTreeFilter outerInstance, AtomicReaderContext context, Bits acceptDocs,
                                    bool hasIndexedLeaves)
                 : base(outerInstance, context, acceptDocs)
@@ -176,7 +177,6 @@ namespace Lucene.Net.Spatial.Prefix
                 this.hasIndexedLeaves = hasIndexedLeaves;
             }
 
-            /// <exception cref="System.IO.IOException"></exception>
             public virtual DocIdSet GetDocIdSet()
             {
                 Debug.Assert(curVNode == null, "Called more than once?");
@@ -192,18 +192,29 @@ namespace Lucene.Net.Spatial.Prefix
                 // all done
                 curVNode = new VNode(null);
                 curVNode.Reset(outerInstance.grid.WorldCell);
+
                 Start();
+
                 AddIntersectingChildren();
-                while (thisTerm != null)
+
+                while (thisTerm != null)//terminates for other reasons too!
                 {
-                    //terminates for other reasons too!
                     //Advance curVNode pointer
                     if (curVNode.children != null)
                     {
                         //-- HAVE CHILDREN: DESCEND
-                        Debug.Assert(curVNode.children.MoveNext());
+
+                        // LUCENENET NOTE: Must call this line before calling MoveNext()
+                        // on the enumerator.
+
                         //if we put it there then it has something
                         PreSiblings(curVNode);
+
+                        // LUCENENET IMPORTANT: Must not call this inline with Debug.Assert
+                        // because the compiler removes Debug.Assert statements in release mode!!
+                        bool hasNext = curVNode.children.MoveNext();
+                        Debug.Assert(hasNext);
+
                         curVNode = curVNode.children.Current;
                     }
                     else
@@ -236,12 +247,11 @@ namespace Lucene.Net.Spatial.Prefix
                     //Seek to curVNode's cell (or skip if termsEnum has moved beyond)
                     curVNodeTerm.Bytes = curVNode.cell.GetTokenBytes();
                     curVNodeTerm.Length = curVNodeTerm.Bytes.Length;
-                    int compare = termsEnum.Comparator.Compare(thisTerm, curVNodeTerm
-                        );
+                    int compare = termsEnum.Comparator.Compare(thisTerm, curVNodeTerm);
                     if (compare > 0)
                     {
                         // leap frog (termsEnum is beyond where we would otherwise seek)
-                        Debug.Assert(!((AtomicReader)context.Reader).Terms(outerInstance.fieldName).Iterator(null).SeekExact(curVNodeTerm), "should be absent");
+                        Debug.Assert(!context.AtomicReader.Terms(outerInstance.fieldName).Iterator(null).SeekExact(curVNodeTerm), "should be absent");
                     }
                     else
                     {
@@ -257,10 +267,9 @@ namespace Lucene.Net.Spatial.Prefix
                             thisTerm = termsEnum.Term();
                             if (seekStatus == TermsEnum.SeekStatus.NOT_FOUND)
                             {
-                                continue;
+                                continue; // leap frog
                             }
                         }
-                        // leap frog
                         // Visit!
                         bool descend = Visit(curVNode.cell);
                         //advance
@@ -302,10 +311,8 @@ namespace Lucene.Net.Spatial.Prefix
                 {
                     //If the next indexed term just adds a leaf marker ('+') to cell,
                     // then add all of those docs
-                    Debug.Assert(StringHelper.StartsWith(thisTerm, curVNodeTerm
-                                     ));
-                    scanCell = outerInstance.grid.GetCell(thisTerm.Bytes, thisTerm.Offset
-                                                       , thisTerm.Length, scanCell);
+                    Debug.Assert(StringHelper.StartsWith(thisTerm, curVNodeTerm));//TODO refactor to use method on curVNode.cell
+                    scanCell = outerInstance.grid.GetCell(thisTerm.Bytes, thisTerm.Offset, thisTerm.Length, scanCell);
                     if (scanCell.Level == cell.Level && scanCell.IsLeaf())
                     {
                         VisitLeaf(scanCell);
@@ -373,73 +380,23 @@ namespace Lucene.Net.Spatial.Prefix
             protected internal virtual void Scan(int scanDetailLevel)
             {
                 for (;
-                    thisTerm != null && StringHelper.StartsWith(thisTerm, curVNodeTerm
-                                            );
+                    thisTerm != null && StringHelper.StartsWith(thisTerm, curVNodeTerm);//TODO refactor to use method on curVNode.cell
                     thisTerm = termsEnum.Next())
                 {
-                    scanCell = outerInstance.grid.GetCell(thisTerm.Bytes, thisTerm.Offset
-                                                       , thisTerm.Length, scanCell);
+                    scanCell = outerInstance.grid.GetCell(thisTerm.Bytes, thisTerm.Offset, thisTerm.Length, scanCell);
+
                     int termLevel = scanCell.Level;
-                    if (termLevel > scanDetailLevel)
+                    if (termLevel < scanDetailLevel)
                     {
-                        continue;
+                        if (scanCell.IsLeaf())
+                            VisitScanned(scanCell);
                     }
-                    if (termLevel == scanDetailLevel || scanCell.IsLeaf())
+                    else if (termLevel == scanDetailLevel)
                     {
-                        VisitScanned(scanCell);
+                        if (!scanCell.IsLeaf())//LUCENE-5529
+                            VisitScanned(scanCell);
                     }
-                }
-            }
-
-            /// <summary>Called first to setup things.</summary>
-            /// <remarks>Called first to setup things.</remarks>
-            /// <exception cref="System.IO.IOException"></exception>
-            protected internal abstract void Start();
-
-            /// <summary>Called last to return the result.</summary>
-            /// <remarks>Called last to return the result.</remarks>
-            /// <exception cref="System.IO.IOException"></exception>
-            protected internal abstract DocIdSet Finish();
-
-            /// <summary>
-            /// Visit an indexed cell returned from
-            /// <see cref="FindSubCellsToVisit(Lucene.Net.Spatial.Prefix.Tree.Cell)">FindSubCellsToVisit(Lucene.Net.Spatial.Prefix.Tree.Cell)
-            /// 	</see>
-            /// .
-            /// </summary>
-            /// <param name="cell">An intersecting cell.</param>
-            /// <returns>
-            /// true to descend to more levels. It is an error to return true
-            /// if cell.level == detailLevel
-            /// </returns>
-            /// <exception cref="System.IO.IOException"></exception>
-            protected internal abstract bool Visit(Cell cell);
-
-            /// <summary>Called after visit() returns true and an indexed leaf cell is found.</summary>
-            /// <remarks>
-            /// Called after visit() returns true and an indexed leaf cell is found. An
-            /// indexed leaf cell means associated documents generally won't be found at
-            /// further detail levels.
-            /// </remarks>
-            /// <exception cref="System.IO.IOException"></exception>
-            protected internal abstract void VisitLeaf(Cell cell);
-
-            /// <summary>The cell is either indexed as a leaf or is the last level of detail.</summary>
-            /// <remarks>
-            /// The cell is either indexed as a leaf or is the last level of detail. It
-            /// might not even intersect the query shape, so be sure to check for that.
-            /// </remarks>
-            /// <exception cref="System.IO.IOException"></exception>
-            protected internal abstract void VisitScanned(Cell cell);
-
-            /// <exception cref="System.IO.IOException"></exception>
-            protected internal virtual void PreSiblings(VNode vNode)
-            {
-            }
-
-            /// <exception cref="System.IO.IOException"></exception>
-            protected internal virtual void PostSiblings(VNode vNode)
-            {
+                }//term loop
             }
 
             #region Nested type: VNodeCellIterator
@@ -451,14 +408,15 @@ namespace Lucene.Net.Spatial.Prefix
             /// </summary>
             private class VNodeCellIterator : IEnumerator<VNode>
             {
-                private readonly VisitorTemplate _enclosing;
+                private readonly VisitorTemplate outerInstance;
                 internal readonly IEnumerator<Cell> cellIter;
 
                 private readonly VNode vNode;
+                private bool first = true;
 
-                internal VNodeCellIterator(VisitorTemplate _enclosing, IEnumerator<Cell> cellIter, VNode vNode)
+                internal VNodeCellIterator(VisitorTemplate outerInstance, IEnumerator<Cell> cellIter, VNode vNode)
                 {
-                    this._enclosing = _enclosing;
+                    this.outerInstance = outerInstance;
                     //term loop
                     this.cellIter = cellIter;
                     this.vNode = vNode;
@@ -475,7 +433,31 @@ namespace Lucene.Net.Spatial.Prefix
 
                 public bool MoveNext()
                 {
-                    return cellIter.MoveNext();
+                    //Debug.Assert(cellIter.Current != null);
+
+                    // LUCENENET NOTE: The consumer of this class calls
+                    // cellIter.MoveNext() before it is instantiated.
+                    // So, the first call here
+                    // to MoveNext() must not move the cursor.
+                    bool result;
+                    if (!first)
+                    {
+                        result = cellIter.MoveNext();
+                    }
+                    else
+                    {
+                        result = true;
+                        first = false;
+                    }
+
+                    // LUCENENET NOTE: Need to skip this call
+                    // if there are no more results because null
+                    // is not allowed
+                    if (result == true)
+                    {
+                        vNode.Reset(cellIter.Current);
+                    }
+                    return result;
                 }
 
                 public void Reset()
@@ -487,8 +469,8 @@ namespace Lucene.Net.Spatial.Prefix
                 {
                     get
                     {
-                        Debug.Assert(cellIter.Current != null);
-                        vNode.Reset(cellIter.Current);
+                        //Debug.Assert(cellIter.Current != null);
+                        //vNode.Reset(cellIter.Current);
                         return vNode;
                     }
                 }
@@ -503,6 +485,56 @@ namespace Lucene.Net.Spatial.Prefix
 
             #endregion
 
+            /// <summary>Called first to setup things.</summary>
+            /// <remarks>Called first to setup things.</remarks>
+            /// <exception cref="System.IO.IOException"></exception>
+            protected internal abstract void Start();
+
+            /// <summary>Called last to return the result.</summary>
+            /// <remarks>Called last to return the result.</remarks>
+            /// <exception cref="System.IO.IOException"></exception>
+            protected internal abstract DocIdSet Finish();
+
+            /// <summary>
+            /// Visit an indexed cell returned from
+            /// <see cref="FindSubCellsToVisit(Lucene.Net.Spatial.Prefix.Tree.Cell)">FindSubCellsToVisit(Lucene.Net.Spatial.Prefix.Tree.Cell)
+            /// 	</see>
+            /// .
+            /// </summary>
+            /// <param name="cell">An intersecting cell.</param>
+            /// <returns>
+            /// true to descend to more levels. It is an error to return true
+            /// if cell.level == detailLevel
+            /// </returns>
+            /// <exception cref="System.IO.IOException"></exception>
+            protected internal abstract bool Visit(Cell cell);
+
+            /// <summary>Called after visit() returns true and an indexed leaf cell is found.</summary>
+            /// <remarks>
+            /// Called after visit() returns true and an indexed leaf cell is found. An
+            /// indexed leaf cell means associated documents generally won't be found at
+            /// further detail levels.
+            /// </remarks>
+            /// <exception cref="System.IO.IOException"></exception>
+            protected internal abstract void VisitLeaf(Cell cell);
+
+            /// <summary>The cell is either indexed as a leaf or is the last level of detail.</summary>
+            /// <remarks>
+            /// The cell is either indexed as a leaf or is the last level of detail. It
+            /// might not even intersect the query shape, so be sure to check for that.
+            /// </remarks>
+            /// <exception cref="System.IO.IOException"></exception>
+            protected internal abstract void VisitScanned(Cell cell);
+
+            /// <exception cref="System.IO.IOException"></exception>
+            protected internal virtual void PreSiblings(VNode vNode)
+            {
+            }
+
+            /// <exception cref="System.IO.IOException"></exception>
+            protected internal virtual void PostSiblings(VNode vNode)
+            {
+            }
             //class VisitorTemplate
         }
 

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
index 76a95b8..ca0eb3d 100644
--- a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
@@ -24,6 +24,7 @@ using Lucene.Net.Spatial.Prefix;
 using Lucene.Net.Spatial.Prefix.Tree;
 using Lucene.Net.Spatial.Queries;
 using Lucene.Net.Util;
+using System.Diagnostics;
 
 namespace Lucene.Net.Spatial.Prefix
 {
@@ -157,8 +158,8 @@ namespace Lucene.Net.Spatial.Prefix
             /// <exception cref="System.IO.IOException"></exception>
             private SmallDocSet GetLeafDocs(Cell leafCell, Bits acceptContains)
             {
-                System.Diagnostics.Debug.Assert(new BytesRef(leafCell.GetTokenBytes()).Equals(termBytes));
-                System.Diagnostics.Debug.Assert(leafCell.Equals(lastLeaf));//don't call for same leaf again
+                Debug.Assert(new BytesRef(leafCell.GetTokenBytes()).Equals(termBytes));
+                Debug.Assert(leafCell.Equals(lastLeaf));//don't call for same leaf again
                 lastLeaf = leafCell;
 
                 if (termsEnum == null)

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
index 4a21ebc..ea36656 100644
--- a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
@@ -181,14 +181,14 @@ namespace Lucene.Net.Spatial.Prefix
         /// 	</remarks>
         internal sealed class CellTokenStream : TokenStream
         {
-            private readonly CharTermAttribute termAtt;
+            private readonly ICharTermAttribute termAtt;
 
             private IEnumerator<Cell> iter = null;
 
             public CellTokenStream(IEnumerator<Cell> tokens)
             {
                 this.iter = tokens;
-                termAtt = AddAttribute<CharTermAttribute>();
+                termAtt = AddAttribute<ICharTermAttribute>();
             }
 
             internal string nextTokenStringNeedingLeaf;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs b/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
index f160688..8af4cd8 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
@@ -22,6 +22,7 @@ using System.Text;
 using Lucene.Net.Spatial.Util;
 using Lucene.Net.Util;
 using Spatial4n.Core.Shapes;
+using Spatial4n.Core.Context;
 
 namespace Lucene.Net.Spatial.Prefix.Tree
 {
@@ -33,6 +34,16 @@ namespace Lucene.Net.Spatial.Prefix.Tree
     /// <lucene.experimental></lucene.experimental>
     public abstract class Cell : IComparable<Cell>
     {
+        /// <summary>
+        /// LUCENENET specific - we need to set the SpatialPrefixTree before calling overridden 
+        /// members of this class, just in case those overridden members require it. This is
+        /// not possible from the subclass because the constructor of the base class runs first.
+        /// So we need to move the reference here and also set it before running the normal constructor
+        /// logic.
+        /// </summary>
+        protected readonly SpatialPrefixTree outerInstance;
+
+
         public const byte LEAF_BYTE = (byte)('+');//NOTE: must sort before letters & numbers
 
         /*
@@ -43,13 +54,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         private int b_off;
         private int b_len;
 
-        /// <summary>Always false for points.</summary>
-        /// <remarks>
-        /// Always false for points. Otherwise, indicate no further sub-cells are going
-        /// to be provided because shapeRel is WITHIN or maxLevels or a detailLevel is
-        /// hit.
-        /// </remarks>
-        protected internal bool leaf;
+        private string token;//this is the only part of equality
 
         /// <summary>
         /// When set via getSubCells(filter), it is the relationship between this cell
@@ -61,10 +66,20 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// </remarks>
         protected internal SpatialRelation shapeRel = SpatialRelation.NULL_VALUE;//set in getSubCells(filter), and via setLeaf().
 
-        private string token;//this is the only part of equality
+        /// <summary>Always false for points.</summary>
+        /// <remarks>
+        /// Always false for points. Otherwise, indicate no further sub-cells are going
+        /// to be provided because shapeRel is WITHIN or maxLevels or a detailLevel is
+        /// hit.
+        /// </remarks>
+        protected internal bool leaf;
 
-        protected internal Cell(string token)
+        protected internal Cell(SpatialPrefixTree outerInstance, string token)
         {
+            // LUCENENET specific - set the outer instance here
+            // because overrides of GetShape() may require it
+            this.outerInstance = outerInstance;
+
             //NOTE: must sort before letters & numbers
             //this is the only part of equality
             this.token = token;
@@ -79,8 +94,12 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             }
         }
 
-        protected internal Cell(byte[] bytes, int off, int len)
+        protected internal Cell(SpatialPrefixTree outerInstance, byte[] bytes, int off, int len)
         {
+            // LUCENENET specific - set the outer instance here
+            // because overrides of GetShape() may require it
+            this.outerInstance = outerInstance;
+
             //ensure any lazy instantiation completes to make this threadsafe
             this.bytes = bytes;
             b_off = off;
@@ -170,7 +189,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             get
             {
                 if (token == null)
-                    throw new InvalidOperationException("Somehow we got a null token");
+                    token = Encoding.UTF8.GetString(bytes, b_off, b_len);
                 return token;
             }
         }
@@ -203,8 +222,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         {
             get
             {
-                return token.Length;
-                //return token != null ? token.Length : b_len;
+                return token != null ? token.Length : b_len;
             }
         }
 

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
index 4ad1fdb..2218ca0 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
@@ -115,19 +115,16 @@ namespace Lucene.Net.Spatial.Prefix.Tree
 
         internal class GhCell : Cell
         {
-            private readonly GeohashPrefixTree _enclosing;
             private IShape shape;
 
-            internal GhCell(GeohashPrefixTree _enclosing, string token)
-                : base(token)
+            internal GhCell(GeohashPrefixTree outerInstance, string token)
+                : base(outerInstance, token)
             {
-                this._enclosing = _enclosing;
             }
 
-            internal GhCell(GeohashPrefixTree _enclosing, byte[] bytes, int off, int len)
-                : base(bytes, off, len)
+            internal GhCell(GeohashPrefixTree outerInstance, byte[] bytes, int off, int len)
+                : base(outerInstance, bytes, off, len)
             {
-                this._enclosing = _enclosing;
             }
 
             public override void Reset(byte[] bytes, int off, int len)
@@ -143,7 +140,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
                 IList<Cell> cells = new List<Cell>(hashes.Length);
                 foreach (string hash in hashes)
                 {
-                    cells.Add(new GhCell(_enclosing, hash));
+                    cells.Add(new GhCell((GeohashPrefixTree)outerInstance, hash));
                 }
                 return cells;
             }
@@ -156,7 +153,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             //8x4
             public override Cell GetSubCell(IPoint p)
             {
-                return _enclosing.GetCell(p, Level + 1);
+                return outerInstance.GetCell(p, Level + 1);
             }
 
             //not performant!
@@ -165,14 +162,14 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             {
                 if (shape == null)
                 {
-                    shape = GeohashUtils.DecodeBoundary(Geohash, _enclosing.ctx);
+                    shape = GeohashUtils.DecodeBoundary(Geohash, outerInstance.ctx);
                 }
                 return shape;
             }
 
             public override IPoint GetCenter()
             {
-                return GeohashUtils.Decode(Geohash, _enclosing.ctx);
+                return GeohashUtils.Decode(Geohash, outerInstance.ctx);
             }
 
             private string Geohash

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
index f99f702..26763cc 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
@@ -231,27 +231,23 @@ namespace Lucene.Net.Spatial.Prefix.Tree
 
         internal class QuadCell : Cell
         {
-            private readonly QuadPrefixTree _enclosing;
             private IShape shape;
 
-            public QuadCell(QuadPrefixTree _enclosing, string token)
-                : base(token)
+            public QuadCell(QuadPrefixTree outerInstance, string token)
+                : base(outerInstance, token)
             {
-                this._enclosing = _enclosing;
             }
 
-            public QuadCell(QuadPrefixTree _enclosing, string token, SpatialRelation shapeRel
+            public QuadCell(QuadPrefixTree outerInstance, string token, SpatialRelation shapeRel
                 )
-                : base(token)
+                : base(outerInstance, token)
             {
-                this._enclosing = _enclosing;
                 this.shapeRel = shapeRel;
             }
 
-            internal QuadCell(QuadPrefixTree _enclosing, byte[] bytes, int off, int len)
-                : base(bytes, off, len)
+            internal QuadCell(QuadPrefixTree outerInstance, byte[] bytes, int off, int len)
+                : base(outerInstance, bytes, off, len)
             {
-                this._enclosing = _enclosing;
             }
 
             public override void Reset(byte[] bytes, int off, int len)
@@ -262,11 +258,12 @@ namespace Lucene.Net.Spatial.Prefix.Tree
 
             protected internal override ICollection<Cell> GetSubCells()
             {
+                QuadPrefixTree outerInstance = (QuadPrefixTree)this.outerInstance;
                 IList<Cell> cells = new List<Cell>(4);
-                cells.Add(new QuadCell(_enclosing, TokenString + "A"));
-                cells.Add(new QuadCell(_enclosing, TokenString + "B"));
-                cells.Add(new QuadCell(_enclosing, TokenString + "C"));
-                cells.Add(new QuadCell(_enclosing, TokenString + "D"));
+                cells.Add(new QuadCell(outerInstance, TokenString + "A"));
+                cells.Add(new QuadCell(outerInstance, TokenString + "B"));
+                cells.Add(new QuadCell(outerInstance, TokenString + "C"));
+                cells.Add(new QuadCell(outerInstance, TokenString + "D"));
                 return cells;
             }
 
@@ -277,7 +274,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
 
             public override Cell GetSubCell(IPoint p)
             {
-                return _enclosing.GetCell(p, Level + 1);
+                return outerInstance.GetCell(p, Level + 1);
             }
 
             //not performant!
@@ -293,22 +290,23 @@ namespace Lucene.Net.Spatial.Prefix.Tree
 
             private IRectangle MakeShape()
             {
+                QuadPrefixTree outerInstance = (QuadPrefixTree)this.outerInstance;
                 string token = TokenString;
-                double xmin = _enclosing.xmin;
-                double ymin = _enclosing.ymin;
+                double xmin = outerInstance.xmin;
+                double ymin = outerInstance.ymin;
                 for (int i = 0; i < token.Length; i++)
                 {
                     char c = token[i];
                     if ('A' == c || 'a' == c)
                     {
-                        ymin += _enclosing.levelH[i];
+                        ymin += outerInstance.levelH[i];
                     }
                     else
                     {
                         if ('B' == c || 'b' == c)
                         {
-                            xmin += _enclosing.levelW[i];
-                            ymin += _enclosing.levelH[i];
+                            xmin += outerInstance.levelW[i];
+                            ymin += outerInstance.levelH[i];
                         }
                         else
                         {
@@ -320,7 +318,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
                                 // nothing really
                                 if ('D' == c || 'd' == c)
                                 {
-                                    xmin += _enclosing.levelW[i];
+                                    xmin += outerInstance.levelW[i];
                                 }
                                 else
                                 {
@@ -335,15 +333,15 @@ namespace Lucene.Net.Spatial.Prefix.Tree
                 double height;
                 if (len > 0)
                 {
-                    width = _enclosing.levelW[len - 1];
-                    height = _enclosing.levelH[len - 1];
+                    width = outerInstance.levelW[len - 1];
+                    height = outerInstance.levelH[len - 1];
                 }
                 else
                 {
-                    width = _enclosing.gridW;
-                    height = _enclosing.gridH;
+                    width = outerInstance.gridW;
+                    height = outerInstance.gridH;
                 }
-                return _enclosing.ctx.MakeRectangle(xmin, xmin + width, ymin, ymin + height);
+                return outerInstance.ctx.MakeRectangle(xmin, xmin + width, ymin, ymin + height);
             }
 
             //QuadCell

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
index 6d97c5c..77f61fb 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
@@ -195,8 +195,8 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// ~20-25% fewer cells.
         /// </param>
         /// <returns>a set of cells (no dups), sorted, immutable, non-null</returns>
-        public virtual IList<Cell> GetCells(IShape shape, int detailLevel
-            , bool inclParents, bool simplify)
+        public virtual IList<Cell> GetCells(IShape shape, int detailLevel, bool inclParents, 
+            bool simplify)
         {
             //TODO consider an on-demand iterator -- it won't build up all cells in memory.
             if (detailLevel > maxLevels)
@@ -208,8 +208,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
                 return GetCells((IPoint)shape, detailLevel, inclParents);
             }
             IList<Cell> cells = new List<Cell>(inclParents ? 4096 : 2048);
-            RecursiveGetCells(WorldCell, shape, detailLevel, inclParents, simplify, cells
-                );
+            RecursiveGetCells(WorldCell, shape, detailLevel, inclParents, simplify, cells);
             return cells;
         }
 
@@ -218,8 +217,9 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// Returns true if cell was added as a leaf. If it wasn't it recursively
         /// descends.
         /// </remarks>
-        private bool RecursiveGetCells(Cell cell, IShape shape, int
-             detailLevel, bool inclParents, bool simplify, IList<Cell> result)
+        private bool RecursiveGetCells(Cell cell, IShape shape, int detailLevel, 
+            bool inclParents, bool simplify, 
+            IList<Cell> result)
         {
             if (cell.Level == detailLevel)
             {
@@ -292,11 +292,11 @@ namespace Lucene.Net.Spatial.Prefix.Tree
 #endif
             }
             string endToken = cell.TokenString;
-            System.Diagnostics.Debug.Assert(endToken.Length == detailLevel);
+            Debug.Assert(endToken.Length == detailLevel);
             IList<Cell> cells = new List<Cell>(detailLevel);
             for (int i = 1; i < detailLevel; i++)
             {
-                cells.Add(GetCell(endToken.Substring(0, i)));
+                cells.Add(GetCell(endToken.Substring(0, i - 0)));
             }
             cells.Add(cell);
             return cells;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
index a1c7011..cca47ec 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
@@ -18,6 +18,7 @@ using System;
 using System.Collections.Generic;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Distance;
+using System.Globalization;
 
 namespace Lucene.Net.Spatial.Prefix.Tree
 {
@@ -74,7 +75,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
                     try
                     {
                         Type c = Type.GetType(cname);
-                        instance = (SpatialPrefixTreeFactory)System.Activator.CreateInstance(c);
+                        instance = (SpatialPrefixTreeFactory)Activator.CreateInstance(c);
                     }
                     catch (Exception e)
                     {
@@ -86,8 +87,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             return instance.NewSPT();
         }
 
-        protected internal virtual void Init(IDictionary<string, string> args, SpatialContext
-             ctx)
+        protected internal virtual void Init(IDictionary<string, string> args, SpatialContext ctx)
         {
             this.args = args;
             this.ctx = ctx;
@@ -99,7 +99,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             string mlStr = args[MaxLevels];
             if (mlStr != null)
             {
-                maxLevels = int.Parse(mlStr);
+                maxLevels = int.Parse(mlStr, CultureInfo.InvariantCulture);
                 return;
             }
             double degrees;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
index 92e4aaf..f3413b2 100644
--- a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
+++ b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
@@ -30,6 +30,10 @@ using Lucene.Net.Support;
 using Lucene.Net.Util;
 using Spatial4n.Core.Context;
 using Spatial4n.Core.Shapes;
+using Lucene.Net.Index;
+using Spatial4n.Core.Io;
+using System.Collections;
+using Lucene.Net.Spatial.Util;
 
 namespace Lucene.Net.Spatial.Serialized
 {
@@ -45,25 +49,418 @@ namespace Lucene.Net.Spatial.Serialized
     /// </summary>
     public class SerializedDVStrategy : SpatialStrategy
     {
-        public SerializedDVStrategy(SpatialContext ctx, string fieldName) : base(ctx, fieldName)
+        /// <summary>
+        /// A cache heuristic for the buf size based on the last shape size.
+        /// </summary>
+        //TODO do we make this non-volatile since it's merely a heuristic?
+        private volatile int indexLastBufSize = 8 * 1024;//8KB default on first run
+
+        /// <summary>
+        /// Constructs the spatial strategy with its mandatory arguments.
+        /// </summary>
+        public SerializedDVStrategy(SpatialContext ctx, string fieldName)
+            : base(ctx, fieldName)
         {
-            throw new NotImplementedException();
         }
 
         public override Field[] CreateIndexableFields(IShape shape)
         {
-            throw new NotImplementedException();
+            int bufSize = Math.Max(128, (int)(this.indexLastBufSize * 1.5));//50% headroom over last
+            ByteArrayOutputStream byteStream = new ByteArrayOutputStream(bufSize);
+            BytesRef bytesRef = new BytesRef();//receiver of byteStream's bytes
+            try
+            {
+                ctx.BinaryCodec.WriteShape(new BinaryWriter(byteStream), shape);
+
+                //this is a hack to avoid redundant byte array copying by byteStream.toByteArray()
+                byteStream.WriteTo(new OutputStreamAnonymousHelper(bytesRef));
+
+
+                //            byteStream.WriteTo(new FilterOutputStream(null/*not used*/) {
+                //    @Override
+                //    public void write(byte[] b, int off, int len) throws IOException
+                //    {
+                //        bytesRef.bytes = b;
+                //        bytesRef.offset = off;
+                //        bytesRef.length = len;
+                //    }
+                //});
+            }
+            catch (IOException e)
+            {
+                throw new ApplicationException(e.Message, e);
+            }
+            this.indexLastBufSize = bytesRef.Length;//cache heuristic
+            return new Field[] { new BinaryDocValuesField(FieldName, bytesRef) };
+        }
+
+        internal class OutputStreamAnonymousHelper : MemoryStream
+        {
+            private readonly BytesRef bytesRef;
+
+            public OutputStreamAnonymousHelper(BytesRef bytesRef)
+            {
+                this.bytesRef = bytesRef;
+            }
+
+            public override void Write(byte[] buffer, int index, int count)
+            {
+                bytesRef.Bytes = buffer;
+                bytesRef.Offset = index;
+                bytesRef.Length = count;
+            }
         }
 
         public override ValueSource MakeDistanceValueSource(IPoint queryPoint, double multiplier)
         {
-            throw new NotImplementedException();
+            //TODO if makeShapeValueSource gets lifted to the top; this could become a generic impl.
+            return new DistanceToShapeValueSource(MakeShapeValueSource(), queryPoint, multiplier, ctx);
         }
 
+        public override ConstantScoreQuery MakeQuery(SpatialArgs args)
+        {
+            throw new NotSupportedException("This strategy can't return a query that operates" +
+                " efficiently. Instead try a Filter or ValueSource.");
+        }
+
+        /// <summary>
+        /// Returns a Filter that should be used with <see cref="FilteredQuery.QUERY_FIRST_FILTER_STRATEGY"/>.
+        /// Use in another manner is likely to result in an <see cref="NotSupportedException"/>
+        /// to prevent misuse because the filter can't efficiently work via iteration.
+        /// </summary>
         public override Filter MakeFilter(SpatialArgs args)
         {
-            throw new NotImplementedException();
+            ValueSource shapeValueSource = MakeShapeValueSource();
+            ShapePredicateValueSource predicateValueSource = new ShapePredicateValueSource(
+                shapeValueSource, args.Operation, args.Shape);
+            return new PredicateValueSourceFilter(predicateValueSource);
         }
-    }
 
+        /**
+        * Provides access to each shape per document as a ValueSource in which
+        * {@link org.apache.lucene.queries.function.FunctionValues#objectVal(int)} returns a {@link
+        * Shape}.
+        */ //TODO raise to SpatialStrategy
+        public virtual ValueSource MakeShapeValueSource()
+        {
+            return new ShapeDocValueSource(this, FieldName, ctx.BinaryCodec);
+        }
+
+        /// <summary>
+        /// This filter only supports returning a DocSet with a GetBits(). If you try to grab the
+        /// iterator then you'll get an UnsupportedOperationException.
+        /// </summary>
+        internal class PredicateValueSourceFilter : Filter
+        {
+            private readonly ValueSource predicateValueSource;//we call boolVal(doc)
+
+            public PredicateValueSourceFilter(ValueSource predicateValueSource)
+            {
+                this.predicateValueSource = predicateValueSource;
+            }
+
+            public override DocIdSet GetDocIdSet(AtomicReaderContext context, Bits acceptDocs)
+            {
+                return new DocIdSetAnonymousHelper(this, context, acceptDocs);
+
+                //      return new DocIdSet()
+                //        {
+                //            @Override
+                //        public DocIdSetIterator iterator() throws IOException
+                //        {
+                //          throw new UnsupportedOperationException(
+                //              "Iteration is too slow; instead try FilteredQuery.QUERY_FIRST_FILTER_STRATEGY");
+                //        //Note that if you're truly bent on doing this, then see FunctionValues.getRangeScorer
+                //    }
+
+                //    @Override
+                //        public Bits bits() throws IOException
+                //    {
+                //        //null Map context -- we simply don't have one. That's ok.
+                //        final FunctionValues predFuncValues = predicateValueSource.getValues(null, context);
+
+                //          return new Bits()
+                //    {
+
+                //        @Override
+                //            public boolean get(int index)
+                //    {
+                //        if (acceptDocs != null && !acceptDocs.get(index))
+                //            return false;
+                //        return predFuncValues.boolVal(index);
+                //    }
+
+                //    @Override
+                //            public int length()
+                //    {
+                //        return context.reader().maxDoc();
+                //    }
+                //};
+                //  }
+                //};
+            }
+
+            internal class DocIdSetAnonymousHelper : DocIdSet
+            {
+                private readonly PredicateValueSourceFilter outerInstance;
+                private readonly AtomicReaderContext context;
+                private readonly Bits acceptDocs;
+
+                public DocIdSetAnonymousHelper(PredicateValueSourceFilter outerInstance, AtomicReaderContext context, Bits acceptDocs)
+                {
+                    this.outerInstance = outerInstance;
+                    this.context = context;
+                    this.acceptDocs = acceptDocs;
+                }
+
+                public override DocIdSetIterator GetIterator()
+                {
+                    throw new NotSupportedException(
+                        "Iteration is too slow; instead try FilteredQuery.QUERY_FIRST_FILTER_STRATEGY");
+                        //Note that if you're truly bent on doing this, then see FunctionValues.getRangeScorer
+                }
+
+                public override Bits GetBits()
+                {
+                    //null Map context -- we simply don't have one. That's ok.
+                    FunctionValues predFuncValues = outerInstance.predicateValueSource.GetValues(null, context);
+
+                    return new BitsAnonymousHelper(this, predFuncValues, context, acceptDocs);
+                }
+
+                internal class BitsAnonymousHelper : Bits
+                {
+                    private readonly DocIdSetAnonymousHelper outerInstance;
+                    private readonly FunctionValues predFuncValues;
+                    private readonly AtomicReaderContext context;
+                    private readonly Bits acceptDocs;
+
+                    public BitsAnonymousHelper(DocIdSetAnonymousHelper outerInstance, FunctionValues predFuncValues, AtomicReaderContext context, Bits acceptDocs)
+                    {
+                        this.outerInstance = outerInstance;
+                        this.predFuncValues = predFuncValues;
+                        this.context = context;
+                        this.acceptDocs = acceptDocs;
+                    }
+
+                    public bool Get(int index)
+                    {
+                        if (acceptDocs != null && !acceptDocs.Get(index))
+                            return false;
+                        return predFuncValues.BoolVal(index);
+                    }
+
+                    public int Length()
+                    {
+                        return context.Reader.MaxDoc;
+                    }
+                }
+            }
+
+            public override bool Equals(object o)
+            {
+                if (this == o) return true;
+                if (o == null || GetType() != o.GetType()) return false;
+
+                PredicateValueSourceFilter that = (PredicateValueSourceFilter)o;
+
+                if (!predicateValueSource.Equals(that.predicateValueSource)) return false;
+
+                return true;
+            }
+
+
+            public override int GetHashCode()
+            {
+                return predicateValueSource.GetHashCode();
+            }
+        }//PredicateValueSourceFilter
+
+        /**
+         * Implements a ValueSource by deserializing a Shape in from BinaryDocValues using BinaryCodec.
+         * @see #makeShapeValueSource()
+         */
+        internal class ShapeDocValueSource : ValueSource
+        {
+            private readonly SerializedDVStrategy outerInstance;
+            private readonly string fieldName;
+            private readonly BinaryCodec binaryCodec;//spatial4n
+
+            internal ShapeDocValueSource(SerializedDVStrategy outerInstance, string fieldName, BinaryCodec binaryCodec)
+            {
+                this.outerInstance = outerInstance;
+                this.fieldName = fieldName;
+                this.binaryCodec = binaryCodec;
+            }
+
+            public override FunctionValues GetValues(IDictionary context, AtomicReaderContext readerContext)
+            {
+                BinaryDocValues docValues = readerContext.AtomicReader.GetBinaryDocValues(fieldName);
+
+                return new FuctionValuesAnonymousHelper(this, docValues);
+
+                //      return new FunctionValues()
+                //{
+                //    int bytesRefDoc = -1;
+                //    BytesRef bytesRef = new BytesRef();//scratch
+
+                //    bool fillBytes(int doc) {
+                //        if (bytesRefDoc != doc)
+                //        {
+                //            docValues.Get(doc, bytesRef);
+                //            bytesRefDoc = doc;
+                //        }
+                //        return bytesRef.length != 0;
+                //    }
+
+                //    @Override
+                //        public boolean exists(int doc)
+                //{
+                //    return fillBytes(doc);
+                //}
+
+                //@Override
+                //        public boolean bytesVal(int doc, BytesRef target)
+                //{
+                //    if (fillBytes(doc))
+                //    {
+                //        target.bytes = bytesRef.bytes;
+                //        target.offset = bytesRef.offset;
+                //        target.length = bytesRef.length;
+                //        return true;
+                //    }
+                //    else
+                //    {
+                //        target.length = 0;
+                //        return false;
+                //    }
+                //}
+
+                //@Override
+                //        public Object objectVal(int docId)
+                //{
+                //    if (!fillBytes(docId))
+                //        return null;
+                //    DataInputStream dataInput = new DataInputStream(
+                //        new ByteArrayInputStream(bytesRef.bytes, bytesRef.offset, bytesRef.length));
+                //    try
+                //    {
+                //        return binaryCodec.readShape(dataInput);
+                //    }
+                //    catch (IOException e)
+                //    {
+                //        throw new RuntimeException(e);
+                //    }
+                //}
+
+                //@Override
+                //        public Explanation explain(int doc)
+                //{
+                //    return new Explanation(Float.NaN, toString(doc));
+                //}
+
+                //@Override
+                //        public String toString(int doc)
+                //{
+                //    return description() + "=" + objectVal(doc);//TODO truncate?
+                //}
+
+                //      };
+            }
+
+            internal class FuctionValuesAnonymousHelper : FunctionValues
+            {
+                private readonly ShapeDocValueSource outerInstance;
+                private readonly BinaryDocValues docValues;
+
+                public FuctionValuesAnonymousHelper(ShapeDocValueSource outerInstance, BinaryDocValues docValues)
+                {
+                    this.outerInstance = outerInstance;
+                    this.docValues = docValues;
+                }
+
+                private int bytesRefDoc = -1;
+                private BytesRef bytesRef = new BytesRef();//scratch
+
+                internal bool FillBytes(int doc)
+                {
+                    if (bytesRefDoc != doc)
+                    {
+                        docValues.Get(doc, bytesRef);
+                        bytesRefDoc = doc;
+                    }
+                    return bytesRef.Length != 0;
+                }
+
+                public override bool Exists(int doc)
+                {
+                    return FillBytes(doc);
+                }
+
+                public override bool BytesVal(int doc, BytesRef target)
+                {
+                    if (FillBytes(doc))
+                    {
+                        target.Bytes = bytesRef.Bytes;
+                        target.Offset = bytesRef.Offset;
+                        target.Length = bytesRef.Length;
+                        return true;
+                    }
+                    else
+                    {
+                        target.Length = 0;
+                        return false;
+                    }
+                }
+
+                public override object ObjectVal(int docId)
+                {
+                    if (!FillBytes(docId))
+                        return null;
+                    BinaryReader dataInput = new BinaryReader(
+                        new MemoryStream(bytesRef.Bytes, bytesRef.Offset, bytesRef.Length));
+                    try
+                    {
+                        return outerInstance.binaryCodec.ReadShape(dataInput);
+                    }
+                    catch (IOException e)
+                    {
+                        throw new ApplicationException(e.Message, e);
+                    }
+                }
+
+                public override string ToString(int doc)
+                {
+                    return outerInstance.Description + "=" + ObjectVal(doc);//TODO truncate?
+                }
+            }
+
+            public override bool Equals(object o)
+            {
+                if (this == o) return true;
+                if (o == null || GetType() != o.GetType()) return false;
+
+                ShapeDocValueSource that = (ShapeDocValueSource)o;
+
+                if (!fieldName.Equals(that.fieldName)) return false;
+
+                return true;
+            }
+
+            public override int GetHashCode()
+            {
+                int result = fieldName.GetHashCode();
+                return result;
+            }
+
+            public override string Description
+            {
+                get
+                {
+                    return "shapeDocVal(" + fieldName + ")";
+                }
+            }
+
+        }//ShapeDocValueSource
+    }
 }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Spatial/Util/DistanceToShapeValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/DistanceToShapeValueSource.cs b/src/Lucene.Net.Spatial/Util/DistanceToShapeValueSource.cs
new file mode 100644
index 0000000..721677c
--- /dev/null
+++ b/src/Lucene.Net.Spatial/Util/DistanceToShapeValueSource.cs
@@ -0,0 +1,151 @@
+\ufeffusing Lucene.Net.Index;
+using Lucene.Net.Queries.Function;
+using Lucene.Net.Queries.Function.DocValues;
+using Lucene.Net.Search;
+using Lucene.Net.Support;
+using Spatial4n.Core.Context;
+using Spatial4n.Core.Distance;
+using Spatial4n.Core.Shapes;
+using System.Collections;
+
+namespace Lucene.Net.Spatial.Util
+{
+    /*
+     * 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.
+     */
+
+    /// <summary>
+    /// The distance from a provided Point to a Point retrieved from a ValueSource via
+    /// <see cref="FunctionValues.ObjectVal(int)"/>. The distance
+    /// is calculated via a <see cref="IDistanceCalculator"/>.
+    /// 
+    /// @lucene.experimental
+    /// </summary>
+    public class DistanceToShapeValueSource : ValueSource
+    {
+        private readonly ValueSource shapeValueSource;
+        private readonly IPoint queryPoint;
+        private readonly double multiplier;
+        private readonly IDistanceCalculator distCalc;
+
+        //TODO if FunctionValues returns NaN; will things be ok?
+        private readonly double nullValue;//computed
+
+        public DistanceToShapeValueSource(ValueSource shapeValueSource, IPoint queryPoint,
+                                          double multiplier, SpatialContext ctx)
+        {
+            this.shapeValueSource = shapeValueSource;
+            this.queryPoint = queryPoint;
+            this.multiplier = multiplier;
+            this.distCalc = ctx.DistCalc;
+            this.nullValue =
+                (ctx.IsGeo ? 180 * multiplier : double.MaxValue);
+        }
+
+        public override string Description
+        {
+            get { return "distance(" + queryPoint + " to " + shapeValueSource.Description + ")*" + multiplier + ")"; }
+        }
+
+        public override void CreateWeight(IDictionary context, IndexSearcher searcher)
+        {
+            shapeValueSource.CreateWeight(context, searcher);
+        }
+
+        internal class DoubleDocValuesAnonymousHelper : DoubleDocValues
+        {
+            private readonly DistanceToShapeValueSource outerInstance;
+            private readonly FunctionValues shapeValues;
+
+            public DoubleDocValuesAnonymousHelper(DistanceToShapeValueSource outerInstance, FunctionValues shapeValues)
+                : base(outerInstance)
+            {
+                this.outerInstance = outerInstance;
+                this.shapeValues = shapeValues;
+            }
+
+            public override double DoubleVal(int doc)
+            {
+                IShape shape = (IShape)shapeValues.ObjectVal(doc);
+                if (shape == null || shape.IsEmpty)
+                    return outerInstance.nullValue;
+                IPoint pt = shape.Center;
+                return outerInstance.distCalc.Distance(outerInstance.queryPoint, pt) * outerInstance.multiplier;
+            }
+
+            public override Explanation Explain(int doc)
+            {
+                Explanation exp = base.Explain(doc);
+                exp.AddDetail(shapeValues.Explain(doc));
+                return exp;
+            }
+        }
+
+        public override FunctionValues GetValues(IDictionary context, AtomicReaderContext readerContext)
+        {
+            FunctionValues shapeValues = shapeValueSource.GetValues(context, readerContext);
+
+            return new DoubleDocValuesAnonymousHelper(this, shapeValues);
+
+            //return new DoubleDocValues(this)
+            //    {
+            //        @Override
+            //  public double doubleVal(int doc)
+            //    {
+            //        Shape shape = (Shape)shapeValues.objectVal(doc);
+            //        if (shape == null || shape.isEmpty())
+            //            return nullValue;
+            //        Point pt = shape.getCenter();
+            //        return distCalc.distance(queryPoint, pt) * multiplier;
+            //    }
+
+            //    @Override
+            //  public Explanation explain(int doc)
+            //    {
+            //        Explanation exp = super.explain(doc);
+            //        exp.addDetail(shapeValues.explain(doc));
+            //        return exp;
+            //    }
+            //};
+        }
+
+        public override bool Equals(object o)
+        {
+            if (this == o) return true;
+            if (o == null || GetType() != o.GetType()) return false;
+
+            DistanceToShapeValueSource that = (DistanceToShapeValueSource)o;
+
+            if (!queryPoint.Equals(that.queryPoint)) return false;
+            if (that.multiplier.CompareTo(multiplier) != 0) return false;
+            if (!shapeValueSource.Equals(that.shapeValueSource)) return false;
+            if (!distCalc.Equals(that.distCalc)) return false;
+
+            return true;
+        }
+
+        public override int GetHashCode()
+        {
+            int result;
+            long temp;
+            result = shapeValueSource.GetHashCode();
+            result = 31 * result + queryPoint.GetHashCode();
+            temp = Number.DoubleToLongBits(multiplier);
+            result = 31 * result + (int)(temp ^ ((long)((ulong)temp) >> 32));
+            return result;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Spatial/Util/ShapePredicateValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ShapePredicateValueSource.cs b/src/Lucene.Net.Spatial/Util/ShapePredicateValueSource.cs
new file mode 100644
index 0000000..fd88f40
--- /dev/null
+++ b/src/Lucene.Net.Spatial/Util/ShapePredicateValueSource.cs
@@ -0,0 +1,146 @@
+\ufeffusing Lucene.Net.Index;
+using Lucene.Net.Queries.Function;
+using Lucene.Net.Queries.Function.DocValues;
+using Lucene.Net.Search;
+using Lucene.Net.Spatial.Queries;
+using Spatial4n.Core.Shapes;
+using System.Collections;
+
+namespace Lucene.Net.Spatial.Util
+{
+    /*
+     * 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.
+     */
+
+    /// <summary>
+    /// A boolean ValueSource that compares a shape from a provided ValueSource with a given Shape and sees
+    /// if it matches a given <see cref="SpatialOperation"/> (the predicate).
+    /// 
+    /// @lucene.experimental
+    /// </summary>
+    public class ShapePredicateValueSource : ValueSource
+    {
+        private readonly ValueSource shapeValuesource;//the left hand side
+        private readonly SpatialOperation op;
+        private readonly IShape queryShape;//the right hand side (constant)
+
+        /// <summary>
+        /// 
+        /// </summary>
+        /// <param name="shapeValuesource">
+        /// Must yield <see cref="IShape"/> instances from it's objectVal(doc). If null
+        /// then the result is false. This is the left-hand (indexed) side.
+        /// </param>
+        /// <param name="op">the predicate</param>
+        /// <param name="queryShape">The shape on the right-hand (query) side.</param>
+        public ShapePredicateValueSource(ValueSource shapeValuesource, SpatialOperation op, IShape queryShape)
+        {
+            this.shapeValuesource = shapeValuesource;
+            this.op = op;
+            this.queryShape = queryShape;
+        }
+
+
+        public override string Description
+        {
+            get { return shapeValuesource + " " + op + " " + queryShape; }
+        }
+
+
+        public override void CreateWeight(IDictionary context, IndexSearcher searcher)
+        {
+            shapeValuesource.CreateWeight(context, searcher);
+        }
+
+        internal class BoolDocValuesAnonymousHelper : BoolDocValues
+        {
+            private readonly ShapePredicateValueSource outerInstance;
+            private readonly FunctionValues shapeValues;
+
+            public BoolDocValuesAnonymousHelper(ShapePredicateValueSource outerInstance, FunctionValues shapeValues)
+                : base(outerInstance)
+            {
+                this.outerInstance = outerInstance;
+                this.shapeValues = shapeValues;
+            }
+
+            public override bool BoolVal(int doc)
+            {
+                IShape indexedShape = (IShape)shapeValues.ObjectVal(doc);
+                if (indexedShape == null)
+                    return false;
+                return outerInstance.op.Evaluate(indexedShape, outerInstance.queryShape);
+            }
+
+            public override Explanation Explain(int doc)
+            {
+                Explanation exp = base.Explain(doc);
+                exp.AddDetail(shapeValues.Explain(doc));
+                return exp;
+            }
+        }
+
+        public override FunctionValues GetValues(IDictionary context, AtomicReaderContext readerContext)
+        {
+            FunctionValues shapeValues = shapeValuesource.GetValues(context, readerContext);
+
+            return new BoolDocValuesAnonymousHelper(this, shapeValues);
+
+            //return new BoolDocValues(this)
+            //    {
+            //        @Override
+            //  public boolean boolVal(int doc)
+            //    {
+            //        Shape indexedShape = (Shape)shapeValues.objectVal(doc);
+            //        if (indexedShape == null)
+            //            return false;
+            //        return op.evaluate(indexedShape, queryShape);
+            //    }
+
+            //    @Override
+            //  public Explanation explain(int doc)
+            //    {
+            //        Explanation exp = super.explain(doc);
+            //        exp.addDetail(shapeValues.explain(doc));
+            //        return exp;
+            //    }
+            //};
+        }
+
+
+        public override bool Equals(object o)
+        {
+            if (this == o) return true;
+            if (o == null || GetType() != o.GetType()) return false;
+
+            ShapePredicateValueSource that = (ShapePredicateValueSource)o;
+
+            if (!shapeValuesource.Equals(that.shapeValuesource)) return false;
+            if (!op.Equals(that.op)) return false;
+            if (!queryShape.Equals(that.queryShape)) return false;
+
+            return true;
+        }
+
+        public override int GetHashCode()
+        {
+            int result = shapeValuesource.GetHashCode();
+            result = 31 * result + op.GetHashCode();
+            result = 31 * result + queryShape.GetHashCode();
+            return result;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
index 5c0d264..dcbe259 100644
--- a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
+++ b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
@@ -125,7 +125,7 @@ namespace Lucene.Net.Spatial.Vector
                 return new ConstantScoreQuery(vsf);
             }
 
-            throw new InvalidOperationException("Only Rectangles and Circles are currently supported, " +
+            throw new NotSupportedException("Only Rectangles and Circles are currently supported, " +
                                             "found [" + shape.GetType().Name + "]"); //TODO
         }
 

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.TestFramework/JavaCompatibility/LuceneTestCase.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.TestFramework/JavaCompatibility/LuceneTestCase.cs b/src/Lucene.Net.TestFramework/JavaCompatibility/LuceneTestCase.cs
index 841feaf..f0dc587 100644
--- a/src/Lucene.Net.TestFramework/JavaCompatibility/LuceneTestCase.cs
+++ b/src/Lucene.Net.TestFramework/JavaCompatibility/LuceneTestCase.cs
@@ -3,6 +3,7 @@ using System.Collections;
 using System.Collections.Generic;
 using System.Linq;
 using NUnit.Framework;
+using System.Diagnostics;
 
 namespace Lucene.Net.Util
 {
@@ -137,5 +138,108 @@ namespace Lucene.Net.Util
         {
             return new HashSet<T>(args);
         }
+
+        protected int randomInt(int max)
+        {
+            return randomIntBetween(0, max);
+        }
+
+        protected int randomIntBetween(int min, int max)
+        {
+            Debug.Assert(max >= min, "max must be >= min: " + min + ", " + max);
+            long range = (long)max - (long)min;
+            if (range < int.MaxValue)
+            {
+                return min + Random().nextInt(1 + (int)range);
+            }
+            else
+            {
+                return toIntExact(min + Random().Next(1 + (int)range));
+            }
+        }
+
+        private static int toIntExact(long value)
+        {
+            if (value > int.MaxValue)
+            {
+                throw new ArithmeticException("Overflow: " + value);
+            }
+            else
+            {
+                return (int)value;
+            }
+        }
+
+        private double nextNextGaussian;
+        private bool haveNextNextGaussian = false;
+
+        /**
+         * Returns the next pseudorandom, Gaussian ("normally") distributed
+         * {@code double} value with mean {@code 0.0} and standard
+         * deviation {@code 1.0} from this random number generator's sequence.
+         * <p>
+         * The general contract of {@code nextGaussian} is that one
+         * {@code double} value, chosen from (approximately) the usual
+         * normal distribution with mean {@code 0.0} and standard deviation
+         * {@code 1.0}, is pseudorandomly generated and returned.
+         *
+         * <p>The method {@code nextGaussian} is implemented by class
+         * {@code Random} as if by a threadsafe version of the following:
+         *  <pre> {@code
+         * private double nextNextGaussian;
+         * private boolean haveNextNextGaussian = false;
+         *
+         * public double nextGaussian() {
+         *   if (haveNextNextGaussian) {
+         *     haveNextNextGaussian = false;
+         *     return nextNextGaussian;
+         *   } else {
+         *     double v1, v2, s;
+         *     do {
+         *       v1 = 2 * nextDouble() - 1;   // between -1.0 and 1.0
+         *       v2 = 2 * nextDouble() - 1;   // between -1.0 and 1.0
+         *       s = v1 * v1 + v2 * v2;
+         *     } while (s >= 1 || s == 0);
+         *     double multiplier = StrictMath.sqrt(-2 * StrictMath.log(s)/s);
+         *     nextNextGaussian = v2 * multiplier;
+         *     haveNextNextGaussian = true;
+         *     return v1 * multiplier;
+         *   }
+         * }}</pre>
+         * This uses the <i>polar method</i> of G. E. P. Box, M. E. Muller, and
+         * G. Marsaglia, as described by Donald E. Knuth in <i>The Art of
+         * Computer Programming</i>, Volume 3: <i>Seminumerical Algorithms</i>,
+         * section 3.4.1, subsection C, algorithm P. Note that it generates two
+         * independent values at the cost of only one call to {@code StrictMath.log}
+         * and one call to {@code StrictMath.sqrt}.
+         *
+         * @return the next pseudorandom, Gaussian ("normally") distributed
+         *         {@code double} value with mean {@code 0.0} and
+         *         standard deviation {@code 1.0} from this random number
+         *         generator's sequence
+         */
+        public double randomGaussian()
+        {
+            // See Knuth, ACP, Section 3.4.1 Algorithm C.
+            if (haveNextNextGaussian)
+            {
+                haveNextNextGaussian = false;
+                return nextNextGaussian;
+            }
+            else
+            {
+                double v1, v2, s;
+                do
+                {
+                    v1 = 2 * Random().NextDouble() - 1; // between -1 and 1
+                    v2 = 2 * Random().NextDouble() - 1; // between -1 and 1
+                    s = v1 * v1 + v2 * v2;
+                } while (s >= 1 || s == 0);
+                double multiplier = Math.Sqrt(-2 * Math.Log(s) / s);
+                nextNextGaussian = v2 * multiplier;
+                haveNextNextGaussian = true;
+                return v1 * multiplier;
+            }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.TestFramework/JavaCompatibility/SystemTypesHelpers.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.TestFramework/JavaCompatibility/SystemTypesHelpers.cs b/src/Lucene.Net.TestFramework/JavaCompatibility/SystemTypesHelpers.cs
index 04abb39..4685998 100644
--- a/src/Lucene.Net.TestFramework/JavaCompatibility/SystemTypesHelpers.cs
+++ b/src/Lucene.Net.TestFramework/JavaCompatibility/SystemTypesHelpers.cs
@@ -203,7 +203,32 @@ namespace Lucene.Net
 
         public static bool removeAll<T>(this ISet<T> s, IEnumerable<T> list)
         {
-            return s.removeAll(list);
+            bool modified = false;
+
+            if (s.Count > list.Count())
+            {
+                for (var i = list.GetEnumerator(); i.MoveNext();)
+                    modified |= s.Remove(i.Current);
+            }
+            else
+            {
+                List<T> toRemove = new List<T>();
+
+                for (var i = s.GetEnumerator(); i.MoveNext();)
+                {
+                    if (list.Contains(i.Current))
+                    {
+                        toRemove.Add(i.Current);
+                    }
+                }
+
+                foreach (var i in toRemove)
+                {
+                    s.Remove(i);
+                    modified = true;
+                }
+            }
+            return modified;
         }
 
         public static void clear<T>(this ISet<T> s)

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/DistanceStrategyTest.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/DistanceStrategyTest.cs b/src/Lucene.Net.Tests.Spatial/DistanceStrategyTest.cs
new file mode 100644
index 0000000..5f0aa7b
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/DistanceStrategyTest.cs
@@ -0,0 +1,155 @@
+\ufeffusing Lucene.Net.Randomized.Generators;
+using Lucene.Net.Spatial.Prefix;
+using Lucene.Net.Spatial.Prefix.Tree;
+using Lucene.Net.Spatial.Serialized;
+using Lucene.Net.Spatial.Vector;
+using Lucene.Net.Support;
+using NUnit.Framework;
+using Spatial4n.Core.Context;
+using Spatial4n.Core.Shapes;
+using System;
+using System.Collections.Generic;
+
+namespace Lucene.Net.Spatial
+{
+    /*
+     * 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.
+     */
+
+    public class DistanceStrategyTest : StrategyTestCase
+    {
+        //@ParametersFactory
+        public static IList<Object[]> Parameters()
+        {
+            List<Object[]> ctorArgs = new List<object[]>();
+
+            SpatialContext ctx = SpatialContext.GEO;
+            SpatialPrefixTree grid;
+            SpatialStrategy strategy;
+
+            grid = new QuadPrefixTree(ctx, 25);
+            strategy = new RecursivePrefixTreeStrategy(grid, "recursive_quad");
+            ctorArgs.Add(new Object[] { new Param(strategy) });
+
+            grid = new GeohashPrefixTree(ctx, 12);
+            strategy = new TermQueryPrefixTreeStrategy(grid, "termquery_geohash");
+            ctorArgs.Add(new Object[] { new Param(strategy) });
+
+            strategy = new PointVectorStrategy(ctx, "pointvector");
+            ctorArgs.Add(new Object[] { new Param(strategy) });
+
+            strategy = new SerializedDVStrategy(ctx, "serialized");
+            ctorArgs.Add(new Object[] { new Param(strategy) });
+
+            return ctorArgs;
+        }
+
+        // this is a hack for clover!
+        public class Param
+        {
+            internal SpatialStrategy strategy;
+
+            internal Param(SpatialStrategy strategy)
+            {
+                this.strategy = strategy;
+            }
+
+
+            public override String ToString()
+            {
+                return strategy.FieldName;
+            }
+        }
+
+        //  private String fieldName;
+
+        //public DistanceStrategyTest(Param param)
+        //{
+        //    SpatialStrategy strategy = param.strategy;
+        //    this.ctx = strategy.SpatialContext;
+        //    this.strategy = strategy;
+        //}
+
+        public override void SetUp()
+        {
+            base.SetUp();
+            SpatialStrategy strategy = ((Param)(RandomInts.RandomFrom(Random(), Parameters()))[0]).strategy;
+            this.ctx = strategy.SpatialContext;
+            this.strategy = strategy;
+        }
+
+
+        protected override bool NeedsDocValues()
+        {
+            return (strategy is SerializedDVStrategy);
+        }
+
+        [Test]
+        public virtual void TestDistanceOrder()
+        {
+            adoc("100", ctx.MakePoint(2, 1));
+            adoc("101", ctx.MakePoint(-1, 4));
+            adoc("103", (IShape)null);//test score for nothing
+            adoc("999", ctx.MakePoint(2, 1));//test deleted
+            Commit();
+            DeleteDoc("999");
+            Commit();
+            //FYI distances are in docid order
+            checkDistValueSource(ctx.MakePoint(4, 3), 2.8274937f, 5.0898066f, 180f);
+            checkDistValueSource(ctx.MakePoint(0, 4), 3.6043684f, 0.9975641f, 180f);
+        }
+
+        [Test]
+        public virtual void TestRecipScore()
+        {
+            IPoint p100 = ctx.MakePoint(2, 1);
+            adoc("100", p100);
+            IPoint p101 = ctx.MakePoint(-1, 4);
+            adoc("101", p101);
+            adoc("103", (IShape)null);//test score for nothing
+            adoc("999", ctx.MakePoint(2, 1));//test deleted
+            Commit();
+            DeleteDoc("999");
+            Commit();
+
+            double dist = ctx.DistCalc.Distance(p100, p101);
+            IShape queryShape = ctx.MakeCircle(2.01, 0.99, dist);
+            CheckValueSource(strategy.MakeRecipDistanceValueSource(queryShape),
+            new float[] { 1.00f, 0.10f, 0f }, 0.09f);
+        }
+
+        // @Override
+        // protected Document newDoc(String id, Shape shape) {
+        //   //called by adoc().  Make compatible with BBoxStrategy.
+        //   if (shape != null && strategy instanceof BBoxStrategy)
+        //     shape = ctx.makeRectangle(shape.getCenter(), shape.getCenter());
+        //   return super.newDoc(id, shape);
+        // }
+
+        internal void checkDistValueSource(IPoint pt, params float[] distances)
+        {
+            float multiplier = (float)Random().NextDouble() * 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);
+        }
+
+    }
+}


[05/26] lucenenet git commit: Updated Lucene.Net.Spatial to use Spatial4n 0.4.1

Posted by ni...@apache.org.
Updated Lucene.Net.Spatial to use Spatial4n 0.4.1


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

Branch: refs/heads/master
Commit: d55479270881f348d1ce80719061b573d67d38b6
Parents: 6c7cc79
Author: Shad Storhaug <sh...@shadstorhaug.com>
Authored: Tue Nov 15 22:17:09 2016 +0700
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Tue Nov 15 22:17:09 2016 +0700

----------------------------------------------------------------------
 Lucene.Net.sln                                  | 79 ++++++++++++++++++--
 .../Lucene.Net.Spatial.csproj                   | 16 +++-
 .../Prefix/AbstractPrefixTreeFilter.cs          |  4 +-
 .../Prefix/AbstractVisitingPrefixTreeFilter.cs  |  2 +-
 .../Prefix/ContainsPrefixTreeFilter.cs          |  6 +-
 .../Prefix/IntersectsPrefixTreeFilter.cs        |  2 +-
 .../Prefix/PointPrefixTreeFieldCacheProvider.cs |  4 +-
 .../Prefix/PrefixTreeStrategy.cs                |  6 +-
 src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs      | 14 ++--
 .../Prefix/Tree/GeohashPrefixTree.cs            | 17 ++---
 .../Prefix/Tree/QuadPrefixTree.cs               | 30 ++++----
 .../Prefix/Tree/SpatialPrefixTree.cs            | 20 ++---
 .../Prefix/WithinPrefixTreeFilter.cs            | 42 +++++------
 src/Lucene.Net.Spatial/Query/SpatialArgs.cs     | 18 ++---
 .../Serialized/SerializedDVStrategy.cs          |  4 +-
 src/Lucene.Net.Spatial/SpatialStrategy.cs       | 16 ++--
 src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs  |  2 +-
 .../Util/ShapeFieldCacheDistanceValueSource.cs  | 16 ++--
 .../Util/ShapeFieldCacheProvider.cs             |  2 +-
 .../Vector/DistanceValueSource.cs               | 12 +--
 .../Vector/PointVectorStrategy.cs               | 58 +++++++-------
 src/Lucene.Net.Spatial/packages.config          |  4 +-
 22 files changed, 227 insertions(+), 147 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/Lucene.Net.sln
----------------------------------------------------------------------
diff --git a/Lucene.Net.sln b/Lucene.Net.sln
index 7fe1c8e..136a3fe 100644
--- a/Lucene.Net.sln
+++ b/Lucene.Net.sln
@@ -69,6 +69,7 @@ EndProject
 Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Lucene.Net.Tests.Analysis.Stempel", "src\Lucene.Net.Tests.Analysis.Stempel\Lucene.Net.Tests.Analysis.Stempel.csproj", "{940A6AB1-F00A-40E2-BC1A-2898EFA8C48F}"
 EndProject
 Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Lucene.Net.Tests.Grouping", "src\Lucene.Net.Tests.Grouping\Lucene.Net.Tests.Grouping.csproj", "{C2349F0D-FB66-4544-9C33-4D87F73C6004}"
+EndProject
 Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Lucene.Net.Spatial", "src\Lucene.Net.Spatial\Lucene.Net.Spatial.csproj", "{35C347F4-24B2-4BE5-8117-A0E3001551CE}"
 EndProject
 Global
@@ -486,140 +487,206 @@ Global
 		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release|x86.ActiveCfg = Release|Any CPU
 		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release|x86.Build.0 = Release|Any CPU
+		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release35|Any CPU.Build.0 = Release|Any CPU
+		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
+		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release35|x86.ActiveCfg = Release|Any CPU
+		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release35|x86.Build.0 = Release|Any CPU
 		{949BA34B-6AE6-4CE3-B578-61E13E4D76BF}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{949BA34B-6AE6-4CE3-B578-61E13E4D76BF}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{949BA34B-6AE6-4CE3-B578-61E13E4D76BF}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{949BA34B-6AE6-4CE3-B578-61E13E4D76BF}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{949BA34B-6AE6-4CE3-B578-61E13E4D76BF}.Debug|x86.ActiveCfg = Debug|Any CPU
+		{949BA34B-6AE6-4CE3-B578-61E13E4D76BF}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{949BA34B-6AE6-4CE3-B578-61E13E4D76BF}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{949BA34B-6AE6-4CE3-B578-61E13E4D76BF}.Debug35|x86.ActiveCfg = Debug|Any CPU
 		{949BA34B-6AE6-4CE3-B578-61E13E4D76BF}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{949BA34B-6AE6-4CE3-B578-61E13E4D76BF}.Release|Any CPU.Build.0 = Release|Any CPU
 		{949BA34B-6AE6-4CE3-B578-61E13E4D76BF}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{949BA34B-6AE6-4CE3-B578-61E13E4D76BF}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{949BA34B-6AE6-4CE3-B578-61E13E4D76BF}.Release|x86.ActiveCfg = Release|Any CPU
+		{949BA34B-6AE6-4CE3-B578-61E13E4D76BF}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{949BA34B-6AE6-4CE3-B578-61E13E4D76BF}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{949BA34B-6AE6-4CE3-B578-61E13E4D76BF}.Release35|x86.ActiveCfg = Release|Any CPU
 		{5719FB4F-BF80-40E5-BACC-37E8E18FCA2E}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{5719FB4F-BF80-40E5-BACC-37E8E18FCA2E}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{5719FB4F-BF80-40E5-BACC-37E8E18FCA2E}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{5719FB4F-BF80-40E5-BACC-37E8E18FCA2E}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{5719FB4F-BF80-40E5-BACC-37E8E18FCA2E}.Debug|x86.ActiveCfg = Debug|Any CPU
+		{5719FB4F-BF80-40E5-BACC-37E8E18FCA2E}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{5719FB4F-BF80-40E5-BACC-37E8E18FCA2E}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{5719FB4F-BF80-40E5-BACC-37E8E18FCA2E}.Debug35|x86.ActiveCfg = Debug|Any CPU
 		{5719FB4F-BF80-40E5-BACC-37E8E18FCA2E}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{5719FB4F-BF80-40E5-BACC-37E8E18FCA2E}.Release|Any CPU.Build.0 = Release|Any CPU
 		{5719FB4F-BF80-40E5-BACC-37E8E18FCA2E}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{5719FB4F-BF80-40E5-BACC-37E8E18FCA2E}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{5719FB4F-BF80-40E5-BACC-37E8E18FCA2E}.Release|x86.ActiveCfg = Release|Any CPU
+		{5719FB4F-BF80-40E5-BACC-37E8E18FCA2E}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{5719FB4F-BF80-40E5-BACC-37E8E18FCA2E}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{5719FB4F-BF80-40E5-BACC-37E8E18FCA2E}.Release35|x86.ActiveCfg = Release|Any CPU
 		{A3A0D943-B91A-4B7A-9FCB-6160EA575D95}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{A3A0D943-B91A-4B7A-9FCB-6160EA575D95}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{A3A0D943-B91A-4B7A-9FCB-6160EA575D95}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{A3A0D943-B91A-4B7A-9FCB-6160EA575D95}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{A3A0D943-B91A-4B7A-9FCB-6160EA575D95}.Debug|x86.ActiveCfg = Debug|Any CPU
 		{A3A0D943-B91A-4B7A-9FCB-6160EA575D95}.Debug|x86.Build.0 = Debug|Any CPU
+		{A3A0D943-B91A-4B7A-9FCB-6160EA575D95}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{A3A0D943-B91A-4B7A-9FCB-6160EA575D95}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{A3A0D943-B91A-4B7A-9FCB-6160EA575D95}.Debug35|x86.ActiveCfg = Debug|Any CPU
 		{A3A0D943-B91A-4B7A-9FCB-6160EA575D95}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{A3A0D943-B91A-4B7A-9FCB-6160EA575D95}.Release|Any CPU.Build.0 = Release|Any CPU
 		{A3A0D943-B91A-4B7A-9FCB-6160EA575D95}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{A3A0D943-B91A-4B7A-9FCB-6160EA575D95}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{A3A0D943-B91A-4B7A-9FCB-6160EA575D95}.Release|x86.ActiveCfg = Release|Any CPU
 		{A3A0D943-B91A-4B7A-9FCB-6160EA575D95}.Release|x86.Build.0 = Release|Any CPU
+		{A3A0D943-B91A-4B7A-9FCB-6160EA575D95}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{A3A0D943-B91A-4B7A-9FCB-6160EA575D95}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{A3A0D943-B91A-4B7A-9FCB-6160EA575D95}.Release35|x86.ActiveCfg = Release|Any CPU
 		{7895E023-EB91-401C-B2B3-754EEC42134B}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{7895E023-EB91-401C-B2B3-754EEC42134B}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{7895E023-EB91-401C-B2B3-754EEC42134B}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{7895E023-EB91-401C-B2B3-754EEC42134B}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{7895E023-EB91-401C-B2B3-754EEC42134B}.Debug|x86.ActiveCfg = Debug|Any CPU
 		{7895E023-EB91-401C-B2B3-754EEC42134B}.Debug|x86.Build.0 = Debug|Any CPU
+		{7895E023-EB91-401C-B2B3-754EEC42134B}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{7895E023-EB91-401C-B2B3-754EEC42134B}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{7895E023-EB91-401C-B2B3-754EEC42134B}.Debug35|x86.ActiveCfg = Debug|Any CPU
 		{7895E023-EB91-401C-B2B3-754EEC42134B}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{7895E023-EB91-401C-B2B3-754EEC42134B}.Release|Any CPU.Build.0 = Release|Any CPU
 		{7895E023-EB91-401C-B2B3-754EEC42134B}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{7895E023-EB91-401C-B2B3-754EEC42134B}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{7895E023-EB91-401C-B2B3-754EEC42134B}.Release|x86.ActiveCfg = Release|Any CPU
 		{7895E023-EB91-401C-B2B3-754EEC42134B}.Release|x86.Build.0 = Release|Any CPU
+		{7895E023-EB91-401C-B2B3-754EEC42134B}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{7895E023-EB91-401C-B2B3-754EEC42134B}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{7895E023-EB91-401C-B2B3-754EEC42134B}.Release35|x86.ActiveCfg = Release|Any CPU
 		{FBD2EB4D-EAC9-409C-A23D-64D27DF23576}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{FBD2EB4D-EAC9-409C-A23D-64D27DF23576}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{FBD2EB4D-EAC9-409C-A23D-64D27DF23576}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{FBD2EB4D-EAC9-409C-A23D-64D27DF23576}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{FBD2EB4D-EAC9-409C-A23D-64D27DF23576}.Debug|x86.ActiveCfg = Debug|Any CPU
 		{FBD2EB4D-EAC9-409C-A23D-64D27DF23576}.Debug|x86.Build.0 = Debug|Any CPU
+		{FBD2EB4D-EAC9-409C-A23D-64D27DF23576}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{FBD2EB4D-EAC9-409C-A23D-64D27DF23576}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{FBD2EB4D-EAC9-409C-A23D-64D27DF23576}.Debug35|x86.ActiveCfg = Debug|Any CPU
 		{FBD2EB4D-EAC9-409C-A23D-64D27DF23576}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{FBD2EB4D-EAC9-409C-A23D-64D27DF23576}.Release|Any CPU.Build.0 = Release|Any CPU
 		{FBD2EB4D-EAC9-409C-A23D-64D27DF23576}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{FBD2EB4D-EAC9-409C-A23D-64D27DF23576}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{FBD2EB4D-EAC9-409C-A23D-64D27DF23576}.Release|x86.ActiveCfg = Release|Any CPU
 		{FBD2EB4D-EAC9-409C-A23D-64D27DF23576}.Release|x86.Build.0 = Release|Any CPU
+		{FBD2EB4D-EAC9-409C-A23D-64D27DF23576}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{FBD2EB4D-EAC9-409C-A23D-64D27DF23576}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{FBD2EB4D-EAC9-409C-A23D-64D27DF23576}.Release35|x86.ActiveCfg = Release|Any CPU
 		{A6511598-3008-4A3B-AE68-2D1DA792CA8A}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{A6511598-3008-4A3B-AE68-2D1DA792CA8A}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{A6511598-3008-4A3B-AE68-2D1DA792CA8A}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{A6511598-3008-4A3B-AE68-2D1DA792CA8A}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{A6511598-3008-4A3B-AE68-2D1DA792CA8A}.Debug|x86.ActiveCfg = Debug|Any CPU
 		{A6511598-3008-4A3B-AE68-2D1DA792CA8A}.Debug|x86.Build.0 = Debug|Any CPU
+		{A6511598-3008-4A3B-AE68-2D1DA792CA8A}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{A6511598-3008-4A3B-AE68-2D1DA792CA8A}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{A6511598-3008-4A3B-AE68-2D1DA792CA8A}.Debug35|x86.ActiveCfg = Debug|Any CPU
 		{A6511598-3008-4A3B-AE68-2D1DA792CA8A}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{A6511598-3008-4A3B-AE68-2D1DA792CA8A}.Release|Any CPU.Build.0 = Release|Any CPU
 		{A6511598-3008-4A3B-AE68-2D1DA792CA8A}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{A6511598-3008-4A3B-AE68-2D1DA792CA8A}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{A6511598-3008-4A3B-AE68-2D1DA792CA8A}.Release|x86.ActiveCfg = Release|Any CPU
 		{A6511598-3008-4A3B-AE68-2D1DA792CA8A}.Release|x86.Build.0 = Release|Any CPU
+		{A6511598-3008-4A3B-AE68-2D1DA792CA8A}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{A6511598-3008-4A3B-AE68-2D1DA792CA8A}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{A6511598-3008-4A3B-AE68-2D1DA792CA8A}.Release35|x86.ActiveCfg = Release|Any CPU
 		{42ECF239-AFC1-427D-921E-B5A277809CF0}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{42ECF239-AFC1-427D-921E-B5A277809CF0}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{42ECF239-AFC1-427D-921E-B5A277809CF0}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{42ECF239-AFC1-427D-921E-B5A277809CF0}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{42ECF239-AFC1-427D-921E-B5A277809CF0}.Debug|x86.ActiveCfg = Debug|Any CPU
 		{42ECF239-AFC1-427D-921E-B5A277809CF0}.Debug|x86.Build.0 = Debug|Any CPU
+		{42ECF239-AFC1-427D-921E-B5A277809CF0}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{42ECF239-AFC1-427D-921E-B5A277809CF0}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{42ECF239-AFC1-427D-921E-B5A277809CF0}.Debug35|x86.ActiveCfg = Debug|Any CPU
 		{42ECF239-AFC1-427D-921E-B5A277809CF0}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{42ECF239-AFC1-427D-921E-B5A277809CF0}.Release|Any CPU.Build.0 = Release|Any CPU
 		{42ECF239-AFC1-427D-921E-B5A277809CF0}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{42ECF239-AFC1-427D-921E-B5A277809CF0}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{42ECF239-AFC1-427D-921E-B5A277809CF0}.Release|x86.ActiveCfg = Release|Any CPU
 		{42ECF239-AFC1-427D-921E-B5A277809CF0}.Release|x86.Build.0 = Release|Any CPU
+		{42ECF239-AFC1-427D-921E-B5A277809CF0}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{42ECF239-AFC1-427D-921E-B5A277809CF0}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{42ECF239-AFC1-427D-921E-B5A277809CF0}.Release35|x86.ActiveCfg = Release|Any CPU
 		{7F9378BF-C88D-46FF-9AE8-5E7D8C0225D3}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{7F9378BF-C88D-46FF-9AE8-5E7D8C0225D3}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{7F9378BF-C88D-46FF-9AE8-5E7D8C0225D3}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{7F9378BF-C88D-46FF-9AE8-5E7D8C0225D3}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{7F9378BF-C88D-46FF-9AE8-5E7D8C0225D3}.Debug|x86.ActiveCfg = Debug|Any CPU
 		{7F9378BF-C88D-46FF-9AE8-5E7D8C0225D3}.Debug|x86.Build.0 = Debug|Any CPU
+		{7F9378BF-C88D-46FF-9AE8-5E7D8C0225D3}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{7F9378BF-C88D-46FF-9AE8-5E7D8C0225D3}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{7F9378BF-C88D-46FF-9AE8-5E7D8C0225D3}.Debug35|x86.ActiveCfg = Debug|Any CPU
 		{7F9378BF-C88D-46FF-9AE8-5E7D8C0225D3}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{7F9378BF-C88D-46FF-9AE8-5E7D8C0225D3}.Release|Any CPU.Build.0 = Release|Any CPU
 		{7F9378BF-C88D-46FF-9AE8-5E7D8C0225D3}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{7F9378BF-C88D-46FF-9AE8-5E7D8C0225D3}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{7F9378BF-C88D-46FF-9AE8-5E7D8C0225D3}.Release|x86.ActiveCfg = Release|Any CPU
 		{7F9378BF-C88D-46FF-9AE8-5E7D8C0225D3}.Release|x86.Build.0 = Release|Any CPU
+		{7F9378BF-C88D-46FF-9AE8-5E7D8C0225D3}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{7F9378BF-C88D-46FF-9AE8-5E7D8C0225D3}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{7F9378BF-C88D-46FF-9AE8-5E7D8C0225D3}.Release35|x86.ActiveCfg = Release|Any CPU
 		{A76DAD88-E3A5-40F9-9114-FACD77BD8265}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{A76DAD88-E3A5-40F9-9114-FACD77BD8265}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{A76DAD88-E3A5-40F9-9114-FACD77BD8265}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{A76DAD88-E3A5-40F9-9114-FACD77BD8265}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{A76DAD88-E3A5-40F9-9114-FACD77BD8265}.Debug|x86.ActiveCfg = Debug|Any CPU
 		{A76DAD88-E3A5-40F9-9114-FACD77BD8265}.Debug|x86.Build.0 = Debug|Any CPU
+		{A76DAD88-E3A5-40F9-9114-FACD77BD8265}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{A76DAD88-E3A5-40F9-9114-FACD77BD8265}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{A76DAD88-E3A5-40F9-9114-FACD77BD8265}.Debug35|x86.ActiveCfg = Debug|Any CPU
 		{A76DAD88-E3A5-40F9-9114-FACD77BD8265}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{A76DAD88-E3A5-40F9-9114-FACD77BD8265}.Release|Any CPU.Build.0 = Release|Any CPU
 		{A76DAD88-E3A5-40F9-9114-FACD77BD8265}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{A76DAD88-E3A5-40F9-9114-FACD77BD8265}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{A76DAD88-E3A5-40F9-9114-FACD77BD8265}.Release|x86.ActiveCfg = Release|Any CPU
 		{A76DAD88-E3A5-40F9-9114-FACD77BD8265}.Release|x86.Build.0 = Release|Any CPU
+		{A76DAD88-E3A5-40F9-9114-FACD77BD8265}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{A76DAD88-E3A5-40F9-9114-FACD77BD8265}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{A76DAD88-E3A5-40F9-9114-FACD77BD8265}.Release35|x86.ActiveCfg = Release|Any CPU
 		{940A6AB1-F00A-40E2-BC1A-2898EFA8C48F}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{940A6AB1-F00A-40E2-BC1A-2898EFA8C48F}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{940A6AB1-F00A-40E2-BC1A-2898EFA8C48F}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{940A6AB1-F00A-40E2-BC1A-2898EFA8C48F}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{940A6AB1-F00A-40E2-BC1A-2898EFA8C48F}.Debug|x86.ActiveCfg = Debug|Any CPU
 		{940A6AB1-F00A-40E2-BC1A-2898EFA8C48F}.Debug|x86.Build.0 = Debug|Any CPU
+		{940A6AB1-F00A-40E2-BC1A-2898EFA8C48F}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{940A6AB1-F00A-40E2-BC1A-2898EFA8C48F}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{940A6AB1-F00A-40E2-BC1A-2898EFA8C48F}.Debug35|x86.ActiveCfg = Debug|Any CPU
 		{940A6AB1-F00A-40E2-BC1A-2898EFA8C48F}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{940A6AB1-F00A-40E2-BC1A-2898EFA8C48F}.Release|Any CPU.Build.0 = Release|Any CPU
 		{940A6AB1-F00A-40E2-BC1A-2898EFA8C48F}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{940A6AB1-F00A-40E2-BC1A-2898EFA8C48F}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{940A6AB1-F00A-40E2-BC1A-2898EFA8C48F}.Release|x86.ActiveCfg = Release|Any CPU
 		{940A6AB1-F00A-40E2-BC1A-2898EFA8C48F}.Release|x86.Build.0 = Release|Any CPU
+		{940A6AB1-F00A-40E2-BC1A-2898EFA8C48F}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{940A6AB1-F00A-40E2-BC1A-2898EFA8C48F}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{940A6AB1-F00A-40E2-BC1A-2898EFA8C48F}.Release35|x86.ActiveCfg = Release|Any CPU
 		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
 		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
 		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Debug|x86.ActiveCfg = Debug|Any CPU
 		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Debug|x86.Build.0 = Debug|Any CPU
+		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Debug35|Any CPU.ActiveCfg = Debug|Any CPU
+		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Debug35|Mixed Platforms.ActiveCfg = Debug|Any CPU
+		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Debug35|x86.ActiveCfg = Debug|Any CPU
 		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Release|Any CPU.ActiveCfg = Release|Any CPU
 		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Release|Any CPU.Build.0 = Release|Any CPU
 		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
 		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Release|Mixed Platforms.Build.0 = Release|Any CPU
 		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Release|x86.ActiveCfg = Release|Any CPU
 		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Release|x86.Build.0 = Release|Any CPU
-		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release35|Any CPU.ActiveCfg = Release|Any CPU
-		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release35|Any CPU.Build.0 = Release|Any CPU
-		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
-		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release35|Mixed Platforms.Build.0 = Release|Any CPU
-		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release35|x86.ActiveCfg = Release|Any CPU
-		{351B75B1-BBD5-4E32-8036-7BED4E0135A6}.Release35|x86.Build.0 = Release|Any CPU
+		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Release35|Any CPU.ActiveCfg = Release|Any CPU
+		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Release35|Mixed Platforms.ActiveCfg = Release|Any CPU
+		{C2349F0D-FB66-4544-9C33-4D87F73C6004}.Release35|x86.ActiveCfg = Release|Any CPU
 		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
 		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Debug|Any CPU.Build.0 = Debug|Any CPU
 		{35C347F4-24B2-4BE5-8117-A0E3001551CE}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj b/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj
index f6669e6..2689008 100644
--- a/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj
+++ b/src/Lucene.Net.Spatial/Lucene.Net.Spatial.csproj
@@ -57,8 +57,20 @@
     <StartupObject />
   </PropertyGroup>
   <ItemGroup>
-    <Reference Include="Spatial4n.Core, Version=0.3.0.0, Culture=neutral, PublicKeyToken=9f9456e1ca16d45e, processorArchitecture=MSIL">
-      <HintPath>..\..\packages\Spatial4n.Core.0.3\lib\net40\Spatial4n.Core.dll</HintPath>
+    <Reference Include="Spatial4n.Core.NTS">
+      <HintPath>..\..\packages\Spatial4n.Core.NTS.0.4.1\lib\net40\Spatial4n.Core.NTS.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="GeoAPI, Version=1.7.4.0, Culture=neutral, PublicKeyToken=a1a0da7def465678, processorArchitecture=MSIL">
+      <HintPath>..\..\packages\GeoAPI.1.7.4\lib\net45\GeoAPI.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="NetTopologySuite, Version=1.14.0.0, Culture=neutral, PublicKeyToken=f580a05016ebada1, processorArchitecture=MSIL">
+      <HintPath>..\..\packages\NetTopologySuite.1.14\lib\net45\NetTopologySuite.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="PowerCollections, Version=1.0.0.0, Culture=neutral, PublicKeyToken=2573bf8a1bdddcd5, processorArchitecture=MSIL">
+      <HintPath>..\..\packages\NetTopologySuite.1.14\lib\net45\PowerCollections.dll</HintPath>
       <Private>True</Private>
     </Reference>
     <Reference Include="System" />

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
index fadab36..0591a35 100644
--- a/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/AbstractPrefixTreeFilter.cs
@@ -29,12 +29,12 @@ namespace Lucene.Net.Spatial.Prefix
     /// </summary>
     public abstract class AbstractPrefixTreeFilter : Filter
     {
-        protected internal readonly Shape queryShape;
+        protected internal readonly IShape queryShape;
         protected internal readonly string fieldName;
         protected internal readonly SpatialPrefixTree grid;//not in equals/hashCode since it's implied for a specific field
         protected internal readonly int detailLevel;
         
-        public AbstractPrefixTreeFilter(Shape queryShape, string fieldName, SpatialPrefixTree grid, int detailLevel)
+        public AbstractPrefixTreeFilter(IShape queryShape, string fieldName, SpatialPrefixTree grid, int detailLevel)
         {
             this.queryShape = queryShape;
             this.fieldName = fieldName;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
index 861c0de..e368a55 100644
--- a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
@@ -45,7 +45,7 @@ namespace Lucene.Net.Spatial.Prefix
 
         protected internal readonly int prefixGridScanLevel;
 
-        public AbstractVisitingPrefixTreeFilter(Shape queryShape, string fieldName, SpatialPrefixTree grid, 
+        public AbstractVisitingPrefixTreeFilter(IShape queryShape, string fieldName, SpatialPrefixTree grid, 
                                                 int detailLevel, int prefixGridScanLevel)
             : base(queryShape, fieldName, grid, detailLevel)
         {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
index c6c448f..017b518 100644
--- a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
@@ -39,7 +39,7 @@ namespace Lucene.Net.Spatial.Prefix
     {
         protected readonly bool multiOverlappingIndexedShapes;
 
-        public ContainsPrefixTreeFilter(Shape queryShape, string fieldName, SpatialPrefixTree grid, int detailLevel, bool multiOverlappingIndexedShapes)
+        public ContainsPrefixTreeFilter(IShape queryShape, string fieldName, SpatialPrefixTree grid, int detailLevel, bool multiOverlappingIndexedShapes)
             : base(queryShape, fieldName, grid, detailLevel)
         {
             this.multiOverlappingIndexedShapes = multiOverlappingIndexedShapes;
@@ -53,7 +53,7 @@ namespace Lucene.Net.Spatial.Prefix
 
         private class ContainsVisitor : BaseTermsEnumTraverser
         {
-            private readonly Shape queryShape;
+            private readonly IShape queryShape;
             private readonly int detailLevel;
             private readonly bool multiOverlappingIndexedShapes;
             private SpatialPrefixTree grid;
@@ -87,7 +87,7 @@ namespace Lucene.Net.Spatial.Prefix
                 SmallDocSet combinedSubResults = null;
 
                 //   Optimization: use null subCellsFilter when we know cell is within the query shape.
-                Shape subCellsFilter = queryShape;
+                IShape subCellsFilter = queryShape;
                 if (cell.Level != 0 && ((cell.GetShapeRel() == null || cell.GetShapeRel() == SpatialRelation.WITHIN)))
                 {
                     subCellsFilter = null;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
index 4b8f25b..3d378e0 100644
--- a/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
@@ -33,7 +33,7 @@ namespace Lucene.Net.Spatial.Prefix
     {
         private readonly bool hasIndexedLeaves;
 
-        public IntersectsPrefixTreeFilter(Shape queryShape, string fieldName, 
+        public IntersectsPrefixTreeFilter(IShape queryShape, string fieldName, 
                                           SpatialPrefixTree grid, int detailLevel,
                                           int prefixGridScanLevel, bool hasIndexedLeaves)
             : base(queryShape, fieldName, grid, detailLevel, prefixGridScanLevel)

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs b/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
index 3459cb7..db36767 100644
--- a/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
+++ b/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
@@ -33,7 +33,7 @@ namespace Lucene.Net.Spatial.Prefix
     /// can only retrieve the central <see cref="Point">Point</see> of the original Shapes.
     /// </summary>
     /// <lucene.internal></lucene.internal>
-    public class PointPrefixTreeFieldCacheProvider : ShapeFieldCacheProvider<Point>
+    public class PointPrefixTreeFieldCacheProvider : ShapeFieldCacheProvider<IPoint>
     {
         internal readonly SpatialPrefixTree grid; //
 
@@ -45,7 +45,7 @@ namespace Lucene.Net.Spatial.Prefix
 
         private Cell scanCell = null;//re-used in readShape to save GC
 
-        protected internal override Point ReadShape(BytesRef term)
+        protected internal override IPoint ReadShape(BytesRef term)
         {
             scanCell = grid.GetCell(term.Bytes, term.Offset, term.Length, scanCell);
             if (scanCell.Level == grid.MaxLevels && !scanCell.IsLeaf())

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
index 5f7c406..003b428 100644
--- a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
@@ -146,13 +146,13 @@ namespace Lucene.Net.Spatial.Prefix
             set { distErrPct = value; }
         }
 
-        public override Field[] CreateIndexableFields(Shape shape)
+        public override Field[] CreateIndexableFields(IShape shape)
         {
             double distErr = SpatialArgs.CalcDistanceFromErrPct(shape, distErrPct, ctx);
             return CreateIndexableFields(shape, distErr);
         }
 
-        public virtual Field[] CreateIndexableFields(Shape shape, double distErr)
+        public virtual Field[] CreateIndexableFields(IShape shape, double distErr)
         {
             int detailLevel = grid.GetLevelForDistance(distErr);
             IList<Cell> cells = grid.GetCells(shape, detailLevel, true, simplifyIndexedCells);
@@ -218,7 +218,7 @@ namespace Lucene.Net.Spatial.Prefix
             }
         }
 
-        public override ValueSource MakeDistanceValueSource(Point queryPoint, double multiplier)
+        public override ValueSource MakeDistanceValueSource(IPoint queryPoint, double multiplier)
         {
             var p = provider.GetOrAdd(FieldName, f => new PointPrefixTreeFieldCacheProvider(grid, FieldName, defaultFieldValuesArrayLen));
             return new ShapeFieldCacheDistanceValueSource(ctx, p, queryPoint, multiplier);

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs b/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
index 319815a..f160688 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
@@ -232,12 +232,12 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// </summary>
         /// <param name="shapeFilter">an optional filter for the returned cells.</param>
         /// <returns>A set of cells (no dups), sorted. Not Modifiable.</returns>
-        public virtual ICollection<Cell> GetSubCells(Shape shapeFilter)
+        public virtual ICollection<Cell> GetSubCells(IShape shapeFilter)
         {
             //Note: Higher-performing subclasses might override to consider the shape filter to generate fewer cells.
-            if (shapeFilter is Point)
+            if (shapeFilter is IPoint)
             {
-                Cell subCell = GetSubCell((Point)shapeFilter);
+                Cell subCell = GetSubCell((IPoint)shapeFilter);
                 subCell.shapeRel = SpatialRelation.CONTAINS;
 #if !NET35
                 return new ReadOnlyCollectionBuilder<Cell>(new[] { subCell }).ToReadOnlyCollection();
@@ -281,7 +281,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// <p/>
         /// Precondition: this.getShape().relate(p) != DISJOINT.
         /// </remarks>
-        public abstract Cell GetSubCell(Point p);
+        public abstract Cell GetSubCell(IPoint p);
 
         //TODO Cell getSubCell(byte b)
         /// <summary>Gets the cells at the next grid cell level that cover this cell.</summary>
@@ -298,11 +298,11 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// </summary>
         public abstract int GetSubCellsSize();
 
-        public abstract Shape GetShape();
+        public abstract IShape GetShape();
 
-        public virtual Point GetCenter()
+        public virtual IPoint GetCenter()
         {
-            return GetShape().GetCenter();
+            return GetShape().Center;
         }
 
         #region Equality overrides

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
index 3452948..4ad1fdb 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
@@ -38,8 +38,8 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         public GeohashPrefixTree(SpatialContext ctx, int maxLevels)
             : base(ctx, maxLevels)
         {
-            Rectangle bounds = ctx.GetWorldBounds();
-            if (bounds.GetMinX() != -180)
+            IRectangle bounds = ctx.WorldBounds;
+            if (bounds.MinX != -180)
             {
                 throw new ArgumentException("Geohash only supports lat-lon world bounds. Got " +
                                             bounds);
@@ -72,10 +72,9 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             return Math.Max(Math.Min(level, maxLevels), 1);
         }
 
-        protected internal override Cell GetCell(Point p, int level)
+        protected internal override Cell GetCell(IPoint p, int level)
         {
-            return new GhCell(this, GeohashUtils.EncodeLatLon(p.GetY(), p.GetX
-                                                                            (), level));
+            return new GhCell(this, GeohashUtils.EncodeLatLon(p.Y, p.X, level));
         }
 
         //args are lat,lon (y,x)
@@ -117,7 +116,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         internal class GhCell : Cell
         {
             private readonly GeohashPrefixTree _enclosing;
-            private Shape shape;
+            private IShape shape;
 
             internal GhCell(GeohashPrefixTree _enclosing, string token)
                 : base(token)
@@ -155,14 +154,14 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             }
 
             //8x4
-            public override Cell GetSubCell(Point p)
+            public override Cell GetSubCell(IPoint p)
             {
                 return _enclosing.GetCell(p, Level + 1);
             }
 
             //not performant!
             //cache
-            public override Shape GetShape()
+            public override IShape GetShape()
             {
                 if (shape == null)
                 {
@@ -171,7 +170,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
                 return shape;
             }
 
-            public override Point GetCenter()
+            public override IPoint GetCenter()
             {
                 return GeohashUtils.Decode(Geohash, _enclosing.ctx);
             }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
index bde41b1..f99f702 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
@@ -53,16 +53,16 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         private readonly double ymid;
         private readonly double ymin;
 
-        public QuadPrefixTree(SpatialContext ctx, Rectangle bounds, int maxLevels)
+        public QuadPrefixTree(SpatialContext ctx, IRectangle bounds, int maxLevels)
             : base(ctx, maxLevels)
         {
             //not really sure how big this should be
             // side
             // number
-            xmin = bounds.GetMinX();
-            xmax = bounds.GetMaxX();
-            ymin = bounds.GetMinY();
-            ymax = bounds.GetMaxY();
+            xmin = bounds.MinX;
+            xmax = bounds.MaxX;
+            ymin = bounds.MinY;
+            ymax = bounds.MaxY;
             levelW = new double[maxLevels];
             levelH = new double[maxLevels];
             levelS = new int[maxLevels];
@@ -90,7 +90,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         }
 
         public QuadPrefixTree(SpatialContext ctx, int maxLevels)
-            : this(ctx, ctx.GetWorldBounds(), maxLevels)
+            : this(ctx, ctx.WorldBounds, maxLevels)
         {
         }
 
@@ -127,10 +127,10 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             return maxLevels;
         }
 
-        protected internal override Cell GetCell(Point p, int level)
+        protected internal override Cell GetCell(IPoint p, int level)
         {
             IList<Cell> cells = new List<Cell>(1);
-            Build(xmid, ymid, 0, cells, new StringBuilder(), ctx.MakePoint(p.GetX(), p.GetY()), level);
+            Build(xmid, ymid, 0, cells, new StringBuilder(), ctx.MakePoint(p.X, p.Y), level);
             return cells[0];
         }
 
@@ -146,7 +146,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         }
 
         private void Build(double x, double y, int level, IList<Cell> matches, StringBuilder
-                                                                                   str, Shape shape, int maxLevel)
+                                                                                   str, IShape shape, int maxLevel)
         {
             Debug.Assert(str.Length == level);
             double w = levelW[level] / 2;
@@ -165,13 +165,13 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         // if we actually use the range property in the query, this could be useful
         private void CheckBattenberg(char c, double cx, double cy, int level, IList<Cell>
                                                                                   matches, StringBuilder str,
-                                     Shape shape, int maxLevel)
+                                     IShape shape, int maxLevel)
         {
             Debug.Assert(str.Length == level);
             double w = levelW[level] / 2;
             double h = levelH[level] / 2;
             int strlen = str.Length;
-            Rectangle rectangle = ctx.MakeRectangle(cx - w, cx + w, cy - h, cy + h);
+            IRectangle rectangle = ctx.MakeRectangle(cx - w, cx + w, cy - h, cy + h);
             SpatialRelation v = shape.Relate(rectangle);
             if (SpatialRelation.CONTAINS == v)
             {
@@ -232,7 +232,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         internal class QuadCell : Cell
         {
             private readonly QuadPrefixTree _enclosing;
-            private Shape shape;
+            private IShape shape;
 
             public QuadCell(QuadPrefixTree _enclosing, string token)
                 : base(token)
@@ -275,14 +275,14 @@ namespace Lucene.Net.Spatial.Prefix.Tree
                 return 4;
             }
 
-            public override Cell GetSubCell(Point p)
+            public override Cell GetSubCell(IPoint p)
             {
                 return _enclosing.GetCell(p, Level + 1);
             }
 
             //not performant!
             //cache
-            public override Shape GetShape()
+            public override IShape GetShape()
             {
                 if (shape == null)
                 {
@@ -291,7 +291,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
                 return shape;
             }
 
-            private Rectangle MakeShape()
+            private IRectangle MakeShape()
             {
                 string token = TokenString;
                 double xmin = _enclosing.xmin;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
index e13245e..6d97c5c 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
@@ -102,10 +102,10 @@ namespace Lucene.Net.Spatial.Prefix.Tree
                 throw new ArgumentException("Level must be in 1 to maxLevels range");
             }
             //TODO cache for each level
-            Cell cell = GetCell(ctx.GetWorldBounds().GetCenter(), level);
-            Rectangle bbox = cell.GetShape().GetBoundingBox();
-            double width = bbox.GetWidth();
-            double height = bbox.GetHeight();
+            Cell cell = GetCell(ctx.WorldBounds.Center, level);
+            IRectangle bbox = cell.GetShape().BoundingBox;
+            double width = bbox.Width;
+            double height = bbox.Height;
             //Use standard cartesian hypotenuse. For geospatial, this answer is larger
             // than the correct one but it's okay to over-estimate.
             return Math.Sqrt(width * width + height * height);
@@ -164,7 +164,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// <code>level</code>
         /// .
         /// </summary>
-        protected internal virtual Cell GetCell(Point p, int level)
+        protected internal virtual Cell GetCell(IPoint p, int level)
         {
             return GetCells(p, level, false)[0];
         }
@@ -195,7 +195,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// ~20-25% fewer cells.
         /// </param>
         /// <returns>a set of cells (no dups), sorted, immutable, non-null</returns>
-        public virtual IList<Cell> GetCells(Shape shape, int detailLevel
+        public virtual IList<Cell> GetCells(IShape shape, int detailLevel
             , bool inclParents, bool simplify)
         {
             //TODO consider an on-demand iterator -- it won't build up all cells in memory.
@@ -203,9 +203,9 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             {
                 throw new ArgumentException("detailLevel > maxLevels");
             }
-            if (shape is Point)
+            if (shape is IPoint)
             {
-                return GetCells((Point)shape, detailLevel, inclParents);
+                return GetCells((IPoint)shape, detailLevel, inclParents);
             }
             IList<Cell> cells = new List<Cell>(inclParents ? 4096 : 2048);
             RecursiveGetCells(WorldCell, shape, detailLevel, inclParents, simplify, cells
@@ -218,7 +218,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// Returns true if cell was added as a leaf. If it wasn't it recursively
         /// descends.
         /// </remarks>
-        private bool RecursiveGetCells(Cell cell, Shape shape, int
+        private bool RecursiveGetCells(Cell cell, IShape shape, int
              detailLevel, bool inclParents, bool simplify, IList<Cell> result)
         {
             if (cell.Level == detailLevel)
@@ -280,7 +280,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// being fast, as its
         /// called repeatedly when incPlarents is true.
         /// </summary>
-        public virtual IList<Cell> GetCells(Point p, int detailLevel, bool inclParents)
+        public virtual IList<Cell> GetCells(IPoint p, int detailLevel, bool inclParents)
         {
             Cell cell = GetCell(p, detailLevel);
             if (!inclParents)

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
index 62d5104..3081bbe 100644
--- a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
@@ -52,7 +52,7 @@ namespace Lucene.Net.Spatial.Prefix
         //  minimal query buffer by looking in a DocValues cache holding a representative
         //  point of each disjoint component of a document's shape(s).
 
-        private readonly Shape bufferedQueryShape;//if null then the whole world
+        private readonly IShape bufferedQueryShape;//if null then the whole world
 
         /// <summary>
         /// See
@@ -64,7 +64,7 @@ namespace Lucene.Net.Spatial.Prefix
         /// where non-matching documents are looked for so they can be excluded. If
         /// -1 is used then the whole world is examined (a good default for correctness).
         /// </summary>
-        public WithinPrefixTreeFilter(Shape queryShape, string fieldName, SpatialPrefixTree grid, 
+        public WithinPrefixTreeFilter(IShape queryShape, string fieldName, SpatialPrefixTree grid, 
                                       int detailLevel, int prefixGridScanLevel, double queryBuffer)
             : base(queryShape, fieldName, grid, detailLevel, prefixGridScanLevel)
         {
@@ -81,7 +81,7 @@ namespace Lucene.Net.Spatial.Prefix
         /// <summary>
         /// Returns a new shape that is larger than shape by at distErr
         /// </summary>
-        protected virtual Shape BufferShape(Shape shape, double distErr)
+        protected virtual IShape BufferShape(IShape shape, double distErr)
         {
             //TODO move this generic code elsewhere?  Spatial4j?
             if (distErr <= 0)
@@ -89,30 +89,30 @@ namespace Lucene.Net.Spatial.Prefix
                 throw new ArgumentException("distErr must be > 0");
             }
             SpatialContext ctx = grid.SpatialContext;
-            if (shape is Point)
+            if (shape is IPoint)
             {
-                return ctx.MakeCircle((Point)shape, distErr);
+                return ctx.MakeCircle((IPoint)shape, distErr);
             }
             else
             {
-                if (shape is Circle)
+                if (shape is ICircle)
                 {
-                    var circle = (Circle)shape;
-                    double newDist = circle.GetRadius() + distErr;
-                    if (ctx.IsGeo() && newDist > 180)
+                    var circle = (ICircle)shape;
+                    double newDist = circle.Radius + distErr;
+                    if (ctx.IsGeo && newDist > 180)
                     {
                         newDist = 180;
                     }
-                    return ctx.MakeCircle(circle.GetCenter(), newDist);
+                    return ctx.MakeCircle(circle.Center, newDist);
                 }
                 else
                 {
-                    Rectangle bbox = shape.GetBoundingBox();
-                    double newMinX = bbox.GetMinX() - distErr;
-                    double newMaxX = bbox.GetMaxX() + distErr;
-                    double newMinY = bbox.GetMinY() - distErr;
-                    double newMaxY = bbox.GetMaxY() + distErr;
-                    if (ctx.IsGeo())
+                    IRectangle bbox = shape.BoundingBox;
+                    double newMinX = bbox.MinX - distErr;
+                    double newMaxX = bbox.MaxX + distErr;
+                    double newMinY = bbox.MinY - distErr;
+                    double newMaxY = bbox.MaxY + distErr;
+                    if (ctx.IsGeo)
                     {
                         if (newMinY < -90)
                         {
@@ -122,7 +122,7 @@ namespace Lucene.Net.Spatial.Prefix
                         {
                             newMaxY = 90;
                         }
-                        if (newMinY == -90 || newMaxY == 90 || bbox.GetWidth() + 2 * distErr > 360)
+                        if (newMinY == -90 || newMaxY == 90 || bbox.Width + 2 * distErr > 360)
                         {
                             newMinX = -180;
                             newMaxX = 180;
@@ -136,10 +136,10 @@ namespace Lucene.Net.Spatial.Prefix
                     else
                     {
                         //restrict to world bounds
-                        newMinX = Math.Max(newMinX, ctx.GetWorldBounds().GetMinX());
-                        newMaxX = Math.Min(newMaxX, ctx.GetWorldBounds().GetMaxX());
-                        newMinY = Math.Max(newMinY, ctx.GetWorldBounds().GetMinY());
-                        newMaxY = Math.Min(newMaxY, ctx.GetWorldBounds().GetMaxY());
+                        newMinX = Math.Max(newMinX, ctx.WorldBounds.MinX);
+                        newMaxX = Math.Min(newMaxX, ctx.WorldBounds.MaxX);
+                        newMinY = Math.Max(newMinY, ctx.WorldBounds.MinY);
+                        newMaxY = Math.Min(newMaxY, ctx.WorldBounds.MaxY);
                     }
                     return ctx.MakeRectangle(newMinX, newMaxX, newMinY, newMaxY);
                 }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialArgs.cs b/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
index 11a3114..64e08aa 100644
--- a/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
+++ b/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
@@ -27,7 +27,7 @@ namespace Lucene.Net.Spatial.Query
 
         public SpatialOperation Operation { get; set; }
 
-        public SpatialArgs(SpatialOperation operation, Shape shape)
+        public SpatialArgs(SpatialOperation operation, IShape shape)
         {
             if (operation == null || shape == null)
                 throw new ArgumentException("operation and shape are required");
@@ -44,24 +44,24 @@ namespace Lucene.Net.Spatial.Query
         /// <param name="distErrPct">0 to 0.5</param>
         /// <param name="ctx">Mandatory</param>
         /// <returns>A distance (in degrees).</returns>
-        public static double CalcDistanceFromErrPct(Shape shape, double distErrPct, SpatialContext ctx)
+        public static double CalcDistanceFromErrPct(IShape shape, double distErrPct, SpatialContext ctx)
         {
             if (distErrPct < 0 || distErrPct > 0.5)
             {
                 throw new ArgumentException("distErrPct " + distErrPct + " must be between [0 to 0.5]", "distErrPct");
             }
-            if (distErrPct == 0 || shape is Point)
+            if (distErrPct == 0 || shape is IPoint)
             {
                 return 0;
             }
-            Rectangle bbox = shape.GetBoundingBox();
+            IRectangle bbox = shape.BoundingBox;
 
             //Compute the distance from the center to a corner.  Because the distance
             // to a bottom corner vs a top corner can vary in a geospatial scenario,
             // take the closest one (greater precision).
-            Point ctr = bbox.GetCenter();
-            double y = (ctr.GetY() >= 0 ? bbox.GetMaxY() : bbox.GetMinY());
-            double diagonalDist = ctx.GetDistCalc().Distance(ctr, bbox.GetMaxX(), y);
+            IPoint ctr = bbox.Center;
+            double y = (ctr.Y >= 0 ? bbox.MaxY : bbox.MinY);
+            double diagonalDist = ctx.DistCalc.Distance(ctr, bbox.MaxX, y);
             return diagonalDist * distErrPct;
         }
 
@@ -86,7 +86,7 @@ namespace Lucene.Net.Spatial.Query
         /// </summary>
         public void Validate()
         {
-            if (Operation.IsTargetNeedsArea() && !Shape.HasArea())
+            if (Operation.IsTargetNeedsArea() && !Shape.HasArea)
             {
                 throw new ArgumentException(Operation + " only supports geometry with area");
             }
@@ -106,7 +106,7 @@ namespace Lucene.Net.Spatial.Query
         // Getters & Setters
         //------------------------------------------------
 
-        public Shape Shape { get; set; }
+        public IShape Shape { get; set; }
 
         /// <summary>
         /// A measure of acceptable error of the shape as a fraction. This effectively

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
index d79c02c..c0692d6 100644
--- a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
+++ b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
@@ -50,12 +50,12 @@ namespace Lucene.Net.Spatial.Serialized
             throw new NotImplementedException();
         }
 
-        public override Field[] CreateIndexableFields(Shape shape)
+        public override Field[] CreateIndexableFields(IShape shape)
         {
             throw new NotImplementedException();
         }
 
-        public override ValueSource MakeDistanceValueSource(Point queryPoint, double multiplier)
+        public override ValueSource MakeDistanceValueSource(IPoint queryPoint, double multiplier)
         {
             throw new NotImplementedException();
         }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/SpatialStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/SpatialStrategy.cs b/src/Lucene.Net.Spatial/SpatialStrategy.cs
index 76f827f..f2de112 100644
--- a/src/Lucene.Net.Spatial/SpatialStrategy.cs
+++ b/src/Lucene.Net.Spatial/SpatialStrategy.cs
@@ -82,7 +82,7 @@ namespace Lucene.Net.Spatial
         /// </summary>
         /// <param name="shape"></param>
         /// <returns>Not null nor will it have null elements.</returns>
-        public abstract Field[] CreateIndexableFields(Shape shape);
+        public abstract Field[] CreateIndexableFields(IShape shape);
 
         /// <summary>
         /// See {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point, double)} called with
@@ -90,7 +90,7 @@ namespace Lucene.Net.Spatial
         /// </summary>
         /// <param name="queryPoint"></param>
         /// <returns></returns>
-        public ValueSource MakeDistanceValueSource(Point queryPoint)
+        public ValueSource MakeDistanceValueSource(IPoint queryPoint)
         {
             return MakeDistanceValueSource(queryPoint, 1.0);
         }
@@ -100,7 +100,7 @@ namespace Lucene.Net.Spatial
         /// indexed shape and {@code queryPoint}.  If there are multiple indexed shapes
         /// then the closest one is chosen.
         /// </summary>
-        public abstract ValueSource MakeDistanceValueSource(Point queryPoint, double multiplier);
+        public abstract ValueSource MakeDistanceValueSource(IPoint queryPoint, double multiplier);
 
         /// <summary>
         /// Make a (ConstantScore) Query based principally on {@link org.apache.lucene.spatial.query.SpatialOperation}
@@ -138,14 +138,14 @@ namespace Lucene.Net.Spatial
         /// </summary>
         /// <param name="queryShape"></param>
         /// <returns></returns>
-        public ValueSource MakeRecipDistanceValueSource(Shape queryShape)
+        public ValueSource MakeRecipDistanceValueSource(IShape queryShape)
         {
-            Rectangle bbox = queryShape.GetBoundingBox();
-            double diagonalDist = ctx.GetDistCalc().Distance(
-                ctx.MakePoint(bbox.GetMinX(), bbox.GetMinY()), bbox.GetMaxX(), bbox.GetMaxY());
+            IRectangle bbox = queryShape.BoundingBox;
+            double diagonalDist = ctx.DistCalc.Distance(
+                ctx.MakePoint(bbox.MinX, bbox.MinY), bbox.MaxX, bbox.MaxY);
             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.Center), 1f, c, c);
         }
 
         public override string ToString()

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs b/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs
index 6c27c54..23d04c4 100644
--- a/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs
+++ b/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs
@@ -25,7 +25,7 @@ namespace Lucene.Net.Spatial.Util
     /// associated with a given docId
     /// </summary>
     /// <typeparam name="T"></typeparam>
-    public class ShapeFieldCache<T> where T : Shape
+    public class ShapeFieldCache<T> where T : IShape
     {
         private readonly IList<T>[] cache;
         public int defaultLength;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs b/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
index d1bcb9e..c5f4af7 100644
--- a/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
+++ b/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
@@ -31,13 +31,13 @@ namespace Lucene.Net.Spatial.Util
     /// </summary>
     public class ShapeFieldCacheDistanceValueSource : ValueSource
     {
-        private readonly ShapeFieldCacheProvider<Point> provider;
+        private readonly ShapeFieldCacheProvider<IPoint> provider;
         private readonly SpatialContext ctx;
-        private readonly Point from;
+        private readonly IPoint from;
         private readonly double multiplier;
 
         public ShapeFieldCacheDistanceValueSource(SpatialContext ctx, 
-            ShapeFieldCacheProvider<Point> provider, Point from, double multiplier)
+            ShapeFieldCacheProvider<IPoint> provider, IPoint from, double multiplier)
         {
             this.ctx = ctx;
             this.from = from;
@@ -48,9 +48,9 @@ namespace Lucene.Net.Spatial.Util
         public class CachedDistanceFunctionValue : FunctionValues
         {
             private readonly ShapeFieldCacheDistanceValueSource enclosingInstance;
-            private readonly ShapeFieldCache<Point> cache;
-            private readonly Point from;
-            private readonly DistanceCalculator calculator;
+            private readonly ShapeFieldCache<IPoint> cache;
+            private readonly IPoint from;
+            private readonly IDistanceCalculator calculator;
             private readonly double nullValue;
 
             public CachedDistanceFunctionValue(AtomicReader reader, ShapeFieldCacheDistanceValueSource enclosingInstance)
@@ -59,8 +59,8 @@ namespace Lucene.Net.Spatial.Util
                 this.enclosingInstance = enclosingInstance;
 
                 from = enclosingInstance.from;
-                calculator = enclosingInstance.ctx.GetDistCalc();
-                nullValue = (enclosingInstance.ctx.IsGeo() ? 180 * enclosingInstance.multiplier : double.MaxValue);
+                calculator = enclosingInstance.ctx.DistCalc;
+                nullValue = (enclosingInstance.ctx.IsGeo ? 180 * enclosingInstance.multiplier : double.MaxValue);
             }
 
             public override float FloatVal(int doc)

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/Util/ShapeFieldCacheProvider.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ShapeFieldCacheProvider.cs b/src/Lucene.Net.Spatial/Util/ShapeFieldCacheProvider.cs
index 51827c3..1a78210 100644
--- a/src/Lucene.Net.Spatial/Util/ShapeFieldCacheProvider.cs
+++ b/src/Lucene.Net.Spatial/Util/ShapeFieldCacheProvider.cs
@@ -39,7 +39,7 @@ namespace Lucene.Net.Spatial.Util
     /// </summary>
     /// <lucene.internal></lucene.internal>
     public abstract class ShapeFieldCacheProvider<T>
-        where T : Shape
+        where T : IShape
     {
         //private Logger log = Logger.GetLogger(GetType().FullName);
 

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs b/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
index d54016f..75785f5 100644
--- a/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
+++ b/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
@@ -32,11 +32,11 @@ namespace Lucene.Net.Spatial.Vector
     /// </summary>
     public class DistanceValueSource : ValueSource
     {
-        private readonly Point from;
+        private readonly IPoint from;
         private readonly PointVectorStrategy strategy;
         private readonly double multiplier;
 
-        public DistanceValueSource(PointVectorStrategy strategy, Point from, double multiplier)
+        public DistanceValueSource(PointVectorStrategy strategy, IPoint from, double multiplier)
         {
             this.strategy = strategy;
             this.from = from;
@@ -75,9 +75,9 @@ namespace Lucene.Net.Spatial.Vector
 
         public class DistanceFunctionValue : FunctionValues
         {
-            private readonly DistanceCalculator calculator;
+            private readonly IDistanceCalculator calculator;
             private readonly DistanceValueSource enclosingInstance;
-            private readonly Point from;
+            private readonly IPoint from;
             private readonly double nullValue;
             private readonly double multiplier;
 
@@ -95,8 +95,8 @@ namespace Lucene.Net.Spatial.Vector
 
                 from = enclosingInstance.from;
                 multiplier = enclosingInstance.multiplier;
-                calculator = enclosingInstance.strategy.SpatialContext.GetDistCalc();
-                nullValue = (enclosingInstance.strategy.SpatialContext.IsGeo() ? 180 * multiplier : double.MaxValue);
+                calculator = enclosingInstance.strategy.SpatialContext.DistCalc;
+                nullValue = (enclosingInstance.strategy.SpatialContext.IsGeo ? 180 * multiplier : double.MaxValue);
             }
 
             public override float FloatVal(int doc)

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
index b4b2e87..25672b9 100644
--- a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
+++ b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
@@ -73,16 +73,16 @@ namespace Lucene.Net.Spatial.Vector
             get { return fieldNameY; }
         }
 
-        public override Field[] CreateIndexableFields(Shape shape)
+        public override Field[] CreateIndexableFields(IShape shape)
         {
-            var point = shape as Point;
+            var point = shape as IPoint;
             if (point != null)
                 return CreateIndexableFields(point);
 
             throw new InvalidOperationException("Can only index Point, not " + shape);
         }
 
-        public Field[] CreateIndexableFields(Point point)
+        public Field[] CreateIndexableFields(IPoint point)
         {
             FieldType doubleFieldType = new FieldType(DoubleField.TYPE_NOT_STORED)
                                             {
@@ -90,13 +90,13 @@ namespace Lucene.Net.Spatial.Vector
                                             };
             var f = new Field[]
                         {
-                            new DoubleField(fieldNameX, point.GetX(), doubleFieldType),
-                            new DoubleField(fieldNameY, point.GetY(), doubleFieldType)
+                            new DoubleField(fieldNameX, point.X, doubleFieldType),
+                            new DoubleField(fieldNameY, point.Y, doubleFieldType)
                         };
             return f;
         }
 
-        public override ValueSource MakeDistanceValueSource(Point queryPoint, double multiplier)
+        public override ValueSource MakeDistanceValueSource(IPoint queryPoint, double multiplier)
         {
             return new DistanceValueSource(this, queryPoint, multiplier);
         }
@@ -108,20 +108,20 @@ namespace Lucene.Net.Spatial.Vector
                                      SpatialOperation.IsWithin))
                 throw new UnsupportedSpatialOperation(args.Operation);
 
-            Shape shape = args.Shape;
-            var bbox = shape as Rectangle;
+            IShape shape = args.Shape;
+            var bbox = shape as IRectangle;
             if (bbox != null)
                 return new ConstantScoreQuery(new QueryWrapperFilter(MakeWithin(bbox)));
 
-            var circle = shape as Circle;
+            var circle = shape as ICircle;
             if (circle != null)
             {
-                bbox = circle.GetBoundingBox();
+                bbox = circle.BoundingBox;
                 var vsf = new ValueSourceFilter(
                     new QueryWrapperFilter(MakeWithin(bbox)),
-                    MakeDistanceValueSource(circle.GetCenter()),
+                    MakeDistanceValueSource(circle.Center),
                     0,
-                    circle.GetRadius());
+                    circle.Radius);
                 return new ConstantScoreQuery(vsf);
             }
 
@@ -134,12 +134,12 @@ namespace Lucene.Net.Spatial.Vector
         {
             // For starters, just limit the bbox
             var shape = args.Shape;
-            if (!(shape is Rectangle || shape is Circle))
+            if (!(shape is IRectangle || shape is ICircle))
                 throw new InvalidOperationException("Only Rectangles and Circles are currently supported, found ["
                     + shape.GetType().Name + "]");//TODO
 
-            Rectangle bbox = shape.GetBoundingBox();
-            if (bbox.GetCrossesDateLine())
+            IRectangle bbox = shape.BoundingBox;
+            if (bbox.CrossesDateLine)
             {
                 throw new InvalidOperationException("Crossing dateline not yet supported");
             }
@@ -160,14 +160,14 @@ namespace Lucene.Net.Spatial.Vector
               SpatialOperation.IsWithin))
             {
                 spatial = MakeWithin(bbox);
-                var circle = args.Shape as Circle;
+                var circle = args.Shape as ICircle;
                 if (circle != null)
                 {
                     // Make the ValueSource
-                    valueSource = MakeDistanceValueSource(shape.GetCenter());
+                    valueSource = MakeDistanceValueSource(shape.Center);
 
                     var vsf = new ValueSourceFilter(
-                        new QueryWrapperFilter(spatial), valueSource, 0, circle.GetRadius());
+                        new QueryWrapperFilter(spatial), valueSource, 0, circle.Radius);
 
                     spatial = new FilteredQuery(new MatchAllDocsQuery(), vsf);
                 }
@@ -188,7 +188,7 @@ namespace Lucene.Net.Spatial.Vector
             }
             else
             {
-                valueSource = MakeDistanceValueSource(shape.GetCenter());
+                valueSource = MakeDistanceValueSource(shape.Center);
             }
             Search.Query spatialRankingQuery = new FunctionQuery(valueSource);
             var bq = new BooleanQuery();
@@ -213,22 +213,22 @@ namespace Lucene.Net.Spatial.Vector
         /// Constructs a query to retrieve documents that fully contain the input envelope.
         /// </summary>
         /// <param name="bbox"></param>
-        private Search.Query MakeWithin(Rectangle bbox)
+        private Search.Query MakeWithin(IRectangle bbox)
         {
             var bq = new BooleanQuery();
             const BooleanClause.Occur MUST = BooleanClause.Occur.MUST;
-            if (bbox.GetCrossesDateLine())
+            if (bbox.CrossesDateLine)
             {
                 //use null as performance trick since no data will be beyond the world bounds
-                bq.Add(RangeQuery(fieldNameX, null /*-180*/, bbox.GetMaxX()), BooleanClause.Occur.SHOULD);
-                bq.Add(RangeQuery(fieldNameX, bbox.GetMinX(), null /*+180*/), BooleanClause.Occur.SHOULD);
+                bq.Add(RangeQuery(fieldNameX, null /*-180*/, bbox.MaxX), BooleanClause.Occur.SHOULD);
+                bq.Add(RangeQuery(fieldNameX, bbox.MinX, null /*+180*/), BooleanClause.Occur.SHOULD);
                 bq.MinimumNumberShouldMatch = 1; //must match at least one of the SHOULD
             }
             else
             {
-                bq.Add(RangeQuery(fieldNameX, bbox.GetMinX(), bbox.GetMaxX()), MUST);
+                bq.Add(RangeQuery(fieldNameX, bbox.MinX, bbox.MaxX), MUST);
             }
-            bq.Add(RangeQuery(fieldNameY, bbox.GetMinY(), bbox.GetMaxY()), MUST);
+            bq.Add(RangeQuery(fieldNameY, bbox.MinY, bbox.MaxY), MUST);
             return bq;
         }
 
@@ -247,12 +247,12 @@ namespace Lucene.Net.Spatial.Vector
         /// Constructs a query to retrieve documents that fully contain the input envelope.
         /// </summary>
         /// <param name="bbox"></param>
-        private Search.Query MakeDisjoint(Rectangle bbox)
+        private Search.Query MakeDisjoint(IRectangle bbox)
         {
-            if (bbox.GetCrossesDateLine())
+            if (bbox.CrossesDateLine)
                 throw new InvalidOperationException("MakeDisjoint doesn't handle dateline cross");
-            Search.Query qX = RangeQuery(fieldNameX, bbox.GetMinX(), bbox.GetMaxX());
-            Search.Query qY = RangeQuery(fieldNameY, bbox.GetMinY(), bbox.GetMaxY());
+            Search.Query qX = RangeQuery(fieldNameX, bbox.MinX, bbox.MaxX);
+            Search.Query qY = RangeQuery(fieldNameY, bbox.MinY, bbox.MaxY);
             var bq = new BooleanQuery { { qX, BooleanClause.Occur.MUST_NOT }, { qY, BooleanClause.Occur.MUST_NOT } };
             return bq;
         }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d5547927/src/Lucene.Net.Spatial/packages.config
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/packages.config b/src/Lucene.Net.Spatial/packages.config
index 27237fc..16ef20c 100644
--- a/src/Lucene.Net.Spatial/packages.config
+++ b/src/Lucene.Net.Spatial/packages.config
@@ -1,4 +1,6 @@
 \ufeff<?xml version="1.0" encoding="utf-8"?>
 <packages>
-  <package id="Spatial4n.Core" version="0.3" targetFramework="net451" />
+  <!--<package id="Spatial4n.Core.NTS" version="0.4.1" targetFramework="net451" />-->
+  <package id="GeoAPI" version="1.7.4" targetFramework="net451" />
+  <package id="NetTopologySuite" version="1.14" targetFramework="net451" />
 </packages>
\ No newline at end of file


[23/26] lucenenet git commit: Spatial: Modified member accessibility to match Lucene + fixed issues that were causing compile warnings

Posted by ni...@apache.org.
Spatial: Modified member accessibility to match Lucene + fixed issues that were causing compile warnings


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

Branch: refs/heads/master
Commit: ad70ad88e1dc9bb948e7255d2c61604f020a6fd9
Parents: aa35405
Author: Shad Storhaug <sh...@shadstorhaug.com>
Authored: Fri Nov 18 01:33:52 2016 +0700
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Fri Nov 25 17:58:29 2016 +0700

----------------------------------------------------------------------
 .../Prefix/IntersectsPrefixTreeFilter.cs               | 13 +++++++++++--
 .../Prefix/WithinPrefixTreeFilter.cs                   |  5 +----
 src/Lucene.Net.Spatial/Query/SpatialArgs.cs            |  2 +-
 src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs      |  3 +--
 .../Serialized/SerializedDVStrategy.cs                 |  4 ++--
 src/Lucene.Net.Spatial/SpatialStrategy.cs              |  2 +-
 src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs       |  4 ++--
 src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs   |  2 +-
 src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs   | 12 ++++++------
 9 files changed, 26 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/ad70ad88/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
index f3d697b..f5c288d 100644
--- a/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
@@ -1,6 +1,7 @@
 using Lucene.Net.Index;
 using Lucene.Net.Search;
 using Lucene.Net.Spatial.Prefix.Tree;
+using Lucene.Net.Support;
 using Lucene.Net.Util;
 using Spatial4n.Core.Shapes;
 
@@ -47,6 +48,16 @@ namespace Lucene.Net.Spatial.Prefix
             return base.Equals(o) && hasIndexedLeaves == ((IntersectsPrefixTreeFilter)o).hasIndexedLeaves;
         }
 
+        /// <summary>
+        /// LUCENENET specific: need to override GetHashCode to 
+        /// prevent a compiler warning and realistically, the hash code
+        /// should work similarly to Equals.
+        /// </summary>
+        public override int GetHashCode()
+        {
+            return HashHelpers.CombineHashCodes(base.GetHashCode(), hasIndexedLeaves.GetHashCode());
+        }
+
         /// <exception cref="System.IO.IOException"></exception>
         public override DocIdSet GetDocIdSet(AtomicReaderContext context, Bits acceptDocs)
         {
@@ -57,13 +68,11 @@ namespace Lucene.Net.Spatial.Prefix
 
         private sealed class _VisitorTemplate_55 : VisitorTemplate
         {
-            private readonly IntersectsPrefixTreeFilter outerInstance;
             private FixedBitSet results;
 
             public _VisitorTemplate_55(IntersectsPrefixTreeFilter outerInstance, AtomicReaderContext context, Bits acceptDocs, bool hasIndexedLeaves)
                 : base(outerInstance, context, acceptDocs, hasIndexedLeaves)
             {
-                this.outerInstance = outerInstance;
             }
 
             protected internal override void Start()

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/ad70ad88/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
index 8021cd0..f78dfb9 100644
--- a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
@@ -153,8 +153,6 @@ namespace Lucene.Net.Spatial.Prefix
 
         private sealed class _VisitorTemplate_121 : VisitorTemplate
         {
-            private readonly WithinPrefixTreeFilter outerInstance;
-
             private FixedBitSet inside;
             private FixedBitSet outside;
             private SpatialRelation visitRelation;
@@ -163,7 +161,6 @@ namespace Lucene.Net.Spatial.Prefix
                 Bits acceptDocs, bool hasIndexedLeaves)
                 : base(outerInstance, context, acceptDocs, hasIndexedLeaves)
             {
-                this.outerInstance = outerInstance;
             }
 
             protected internal override void Start()
@@ -181,7 +178,7 @@ namespace Lucene.Net.Spatial.Prefix
             protected internal override IEnumerator<Cell> FindSubCellsToVisit(Cell cell)
             {
                 //use buffered query shape instead of orig.  Works with null too.
-                return cell.GetSubCells(outerInstance.bufferedQueryShape).GetEnumerator();
+                return cell.GetSubCells(((WithinPrefixTreeFilter)outerInstance).bufferedQueryShape).GetEnumerator();
             }
 
             protected internal override bool Visit(Cell cell)

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/ad70ad88/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialArgs.cs b/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
index d5921c8..15dc0f1 100644
--- a/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
+++ b/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
@@ -87,7 +87,7 @@ namespace Lucene.Net.Spatial.Queries
         /// <summary>
         /// Check if the arguments make sense -- throw an exception if not
         /// </summary>
-        public void Validate()
+        public virtual void Validate()
         {
             if (Operation.IsTargetNeedsArea && !Shape.HasArea)
             {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/ad70ad88/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs b/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
index 280ce8e..db0f4ca 100644
--- a/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
+++ b/src/Lucene.Net.Spatial/Query/SpatialArgsParser.cs
@@ -73,7 +73,7 @@ namespace Lucene.Net.Spatial.Queries
         /// <exception cref="ArgumentException">if the parameters don't make sense or an add-on parameter is unknown</exception>
         /// <exception cref="ParseException">If there is a problem parsing the string</exception>
         /// <exception cref="InvalidShapeException">When the coordinates are invalid for the shape</exception>
-        public SpatialArgs Parse(string v, SpatialContext ctx)
+        public virtual SpatialArgs Parse(string v, SpatialContext ctx)
         {
             int idx = v.IndexOf('(');
             int edx = v.LastIndexOf(')');
@@ -174,6 +174,5 @@ namespace Lucene.Net.Spatial.Queries
 
             return map;
         }
-
     }
 }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/ad70ad88/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
index 33926ee..d85ce1e 100644
--- a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
+++ b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
@@ -191,14 +191,14 @@ namespace Lucene.Net.Spatial.Serialized
                         this.acceptDocs = acceptDocs;
                     }
 
-                    public bool Get(int index)
+                    public virtual bool Get(int index)
                     {
                         if (acceptDocs != null && !acceptDocs.Get(index))
                             return false;
                         return predFuncValues.BoolVal(index);
                     }
 
-                    public int Length()
+                    public virtual int Length()
                     {
                         return context.Reader.MaxDoc;
                     }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/ad70ad88/src/Lucene.Net.Spatial/SpatialStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/SpatialStrategy.cs b/src/Lucene.Net.Spatial/SpatialStrategy.cs
index cc679e2..a9bbde1 100644
--- a/src/Lucene.Net.Spatial/SpatialStrategy.cs
+++ b/src/Lucene.Net.Spatial/SpatialStrategy.cs
@@ -52,7 +52,7 @@ namespace Lucene.Net.Spatial
     public abstract class SpatialStrategy
     {
         protected readonly SpatialContext ctx;
-        protected readonly string fieldName;
+        private readonly string fieldName;
 
         /// <summary>
         /// Constructs the spatial strategy with its mandatory arguments.

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/ad70ad88/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs b/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
index 32a5bd9..1abfa09 100644
--- a/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
+++ b/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
@@ -32,8 +32,8 @@ namespace Lucene.Net.Spatial.Util
     {
         //TODO see https://issues.apache.org/jira/browse/LUCENE-4251  (move out of spatial & improve)
 
-        readonly Filter startingFilter;
-        readonly ValueSource source;
+        internal readonly Filter startingFilter;
+        internal readonly ValueSource source;
         public readonly double min;
         public readonly double max;
 

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/ad70ad88/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs b/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
index 5a4cbd0..6f1dfb6 100644
--- a/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
+++ b/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
@@ -34,8 +34,8 @@ namespace Lucene.Net.Spatial.Vector
     /// </summary>
     public class DistanceValueSource : ValueSource
     {
-        private readonly IPoint from;
         private readonly PointVectorStrategy strategy;
+        private readonly IPoint from;
         private readonly double multiplier;
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/ad70ad88/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
index 19c9dc8..3526291 100644
--- a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
+++ b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
@@ -44,7 +44,7 @@ namespace Lucene.Net.Spatial.Vector
         private readonly string fieldNameX;
         private readonly string fieldNameY;
 
-        public int precisionStep = 8; // same as solr default
+        private int precisionStep = 8; // same as solr default
 
         public PointVectorStrategy(SpatialContext ctx, string fieldNamePrefix)
             : base(ctx, fieldNamePrefix)
@@ -53,7 +53,7 @@ namespace Lucene.Net.Spatial.Vector
             this.fieldNameY = fieldNamePrefix + SUFFIX_Y;
         }
 
-        public int PrecisionStep
+        public virtual int PrecisionStep
         {
             set
             {
@@ -63,12 +63,12 @@ namespace Lucene.Net.Spatial.Vector
             }
         }
 
-        public string FieldNameX
+        internal virtual string FieldNameX
         {
             get { return fieldNameX; }
         }
 
-        public string FieldNameY
+        internal virtual string FieldNameY
         {
             get { return fieldNameY; }
         }
@@ -85,7 +85,7 @@ namespace Lucene.Net.Spatial.Vector
         /// <summary>
         /// See <see cref="CreateIndexableFields(IShape)"/>
         /// </summary>
-        public Field[] CreateIndexableFields(IPoint point)
+        public virtual Field[] CreateIndexableFields(IPoint point)
         {
             FieldType doubleFieldType = new FieldType(DoubleField.TYPE_NOT_STORED)
             {
@@ -147,7 +147,7 @@ namespace Lucene.Net.Spatial.Vector
         }
 
         //TODO this is basically old code that hasn't been verified well and should probably be removed
-        public Query MakeQueryDistanceScore(SpatialArgs args)
+        public virtual Query MakeQueryDistanceScore(SpatialArgs args)
         {
             // For starters, just limit the bbox
             var shape = args.Shape;


[14/26] lucenenet git commit: Added missing Spatial classes, ported Spatial tests, and fixed several bugs

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/SpatialTestCase.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/SpatialTestCase.cs b/src/Lucene.Net.Tests.Spatial/SpatialTestCase.cs
new file mode 100644
index 0000000..ebc5374
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/SpatialTestCase.cs
@@ -0,0 +1,248 @@
+\ufeffusing Lucene.Net.Analysis;
+using Lucene.Net.Codecs.Lucene45;
+using Lucene.Net.Documents;
+using Lucene.Net.Index;
+using Lucene.Net.Search;
+using Lucene.Net.Util;
+using Spatial4n.Core.Context;
+using Spatial4n.Core.Shapes;
+using System;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.IO;
+using System.Text;
+
+namespace Lucene.Net.Spatial
+{
+    /*
+     * 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.
+     */
+
+    /// <summary>
+    /// A base test class for spatial lucene. It's mostly Lucene generic.
+    /// </summary>
+    public abstract class SpatialTestCase : LuceneTestCase
+    {
+        private DirectoryReader indexReader;
+        protected RandomIndexWriter indexWriter;
+        private Store.Directory directory;
+        protected IndexSearcher indexSearcher;
+
+        protected SpatialContext ctx;//subclass must initialize
+
+        public override void SetUp()
+        {
+            base.SetUp();
+
+            directory = NewDirectory();
+            Random random = Random();
+            indexWriter = new RandomIndexWriter(random, directory, newIndexWriterConfig(random));
+            indexReader = indexWriter.Reader;
+            indexSearcher = NewSearcher(indexReader);
+        }
+
+        protected virtual IndexWriterConfig newIndexWriterConfig(Random random)
+        {
+            IndexWriterConfig indexWriterConfig = NewIndexWriterConfig(random, TEST_VERSION_CURRENT, new MockAnalyzer(random));
+            //TODO can we randomly choose a doc-values supported format?
+            if (NeedsDocValues())
+                indexWriterConfig.SetCodec(TestUtil.AlwaysDocValuesFormat(new Lucene45DocValuesFormat())); ;
+            return indexWriterConfig;
+        }
+
+        protected virtual bool NeedsDocValues()
+        {
+            return false;
+        }
+
+        public override void TearDown()
+        {
+            IOUtils.Close(indexWriter, indexReader, directory);
+            base.TearDown();
+        }
+
+        // ================================================= Helper Methods ================================================
+
+        protected virtual void AddDocument(Document doc)
+        {
+            indexWriter.AddDocument(doc);
+        }
+
+        protected virtual void addDocumentsAndCommit(List<Document> documents)
+        {
+            foreach (Document document in documents)
+            {
+                indexWriter.AddDocument(document);
+            }
+            Commit();
+        }
+
+        protected virtual void DeleteAll()
+        {
+            indexWriter.DeleteAll();
+        }
+
+        protected virtual void Commit()
+        {
+            indexWriter.Commit();
+            IOUtils.Close(indexReader);
+            indexReader = indexWriter.Reader;
+            indexSearcher = NewSearcher(indexReader);
+        }
+
+        protected virtual void VerifyDocumentsIndexed(int numDocs)
+        {
+            assertEquals(numDocs, indexReader.NumDocs);
+        }
+
+        protected virtual SearchResults executeQuery(Query query, int numDocs)
+        {
+            try
+            {
+                TopDocs topDocs = indexSearcher.Search(query, numDocs);
+
+                List<SearchResult> results = new List<SearchResult>();
+                foreach (ScoreDoc scoreDoc in topDocs.ScoreDocs)
+                {
+                    results.Add(new SearchResult(scoreDoc.Score, indexSearcher.Doc(scoreDoc.Doc)));
+                }
+                return new SearchResults(topDocs.TotalHits, results);
+            }
+            catch (IOException ioe)
+            {
+                throw new ApplicationException("IOException thrown while executing query", ioe);
+            }
+        }
+
+        protected virtual Spatial4n.Core.Shapes.IPoint randomPoint()
+        {
+            IRectangle WB = ctx.WorldBounds;
+            return ctx.MakePoint(
+                randomIntBetween((int)WB.MinX, (int)WB.MaxX),
+                randomIntBetween((int)WB.MinY, (int)WB.MaxY));
+        }
+
+        protected virtual IRectangle randomRectangle()
+        {
+            IRectangle WB = ctx.WorldBounds;
+            int rW = (int)randomGaussianMeanMax(10, WB.Width);
+            double xMin = randomIntBetween((int)WB.MinX, (int)WB.MaxX - rW);
+            double xMax = xMin + rW;
+
+            int yH = (int)randomGaussianMeanMax(Math.Min(rW, WB.Height), WB.Height);
+            double yMin = randomIntBetween((int)WB.MinY, (int)WB.MaxY - yH);
+            double yMax = yMin + yH;
+
+            return ctx.MakeRectangle(xMin, xMax, yMin, yMax);
+        }
+
+        private double randomGaussianMinMeanMax(double min, double mean, double max)
+        {
+            Debug.Assert(mean > min);
+            return randomGaussianMeanMax(mean - min, max - min) + min;
+        }
+
+        /**
+         * Within one standard deviation (68% of the time) the result is "close" to
+         * mean. By "close": when greater than mean, it's the lesser of 2*mean or half
+         * way to max, when lesser than mean, it's the greater of max-2*mean or half
+         * way to 0. The other 32% of the time it's in the rest of the range, touching
+         * either 0 or max but never exceeding.
+         */
+        private double randomGaussianMeanMax(double mean, double max)
+        {
+            // DWS: I verified the results empirically
+            Debug.Assert(mean <= max && mean >= 0);
+            double g = randomGaussian();
+            double mean2 = mean;
+            double flip = 1;
+            if (g < 0)
+            {
+                mean2 = max - mean;
+                flip = -1;
+                g *= -1;
+            }
+            // pivot is the distance from mean2 towards max where the boundary of
+            // 1 standard deviation alters the calculation
+            double pivotMax = max - mean2;
+            double pivot = Math.Min(mean2, pivotMax / 2);//from 0 to max-mean2
+            Debug.Assert(pivot >= 0 && pivotMax >= pivot && g >= 0);
+            double pivotResult;
+            if (g <= 1)
+                pivotResult = pivot * g;
+            else
+                pivotResult = Math.Min(pivotMax, (g - 1) * (pivotMax - pivot) + pivot);
+
+            return mean + flip * pivotResult;
+        }
+
+        // ================================================= Inner Classes =================================================
+
+        protected class SearchResults
+        {
+
+            public int numFound;
+            public List<SearchResult> results;
+
+            public SearchResults(int numFound, List<SearchResult> results)
+            {
+                this.numFound = numFound;
+                this.results = results;
+            }
+
+            public StringBuilder toDebugString()
+            {
+                StringBuilder str = new StringBuilder();
+                str.append("found: ").append(numFound).append('[');
+                foreach (SearchResult r in results)
+                {
+                    string id = r.GetId();
+                    str.append(id).append(", ");
+                }
+                str.append(']');
+                return str;
+            }
+
+            public override string ToString()
+            {
+                return "[found:" + numFound + " " + results + "]";
+            }
+        }
+
+        protected class SearchResult
+        {
+
+            public float score;
+            public Document document;
+
+            public SearchResult(float score, Document document)
+            {
+                this.score = score;
+                this.document = document;
+            }
+
+            public string GetId()
+            {
+                return document.Get("id");
+            }
+
+            public override string ToString()
+            {
+                return "[" + score + "=" + document + "]";
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/SpatialTestData.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/SpatialTestData.cs b/src/Lucene.Net.Tests.Spatial/SpatialTestData.cs
new file mode 100644
index 0000000..231c1cd
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/SpatialTestData.cs
@@ -0,0 +1,78 @@
+\ufeffusing Spatial4n.Core.Context;
+using Spatial4n.Core.Exceptions;
+using Spatial4n.Core.Shapes;
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+
+namespace Lucene.Net.Spatial
+{
+    /*
+     * 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.
+     */
+
+    /// <summary>
+    /// This class is modelled after SpatialTestQuery.
+    /// Before Lucene 4.7, this was a bit different in Spatial4n as SampleData & SampleDataReader.
+    /// </summary>
+    public class SpatialTestData
+    {
+        public String id;
+        public String name;
+        public IShape shape;
+
+        /** Reads the stream, consuming a format that is a tab-separated values of 3 columns:
+         * an "id", a "name" and the "shape".  Empty lines and lines starting with a '#' are skipped.
+         * The stream is closed.
+         */
+        public static IEnumerator<SpatialTestData> GetTestData(Stream @in, SpatialContext ctx)
+        {
+            List<SpatialTestData> results = new List<SpatialTestData>();
+            TextReader bufInput = new StreamReader(@in, Encoding.UTF8);
+            try
+            {
+                String line;
+                while ((line = bufInput.ReadLine()) != null)
+                {
+                    if (line.Length == 0 || line[0] == '#')
+                        continue;
+
+                    SpatialTestData data = new SpatialTestData();
+                    String[] vals = line.Split(new char[] { '\t' }, StringSplitOptions.RemoveEmptyEntries);
+                    if (vals.Length != 3)
+                        throw new ArgumentException("bad format; expecting 3 tab-separated values for line: " + line);
+                    data.id = vals[0];
+                    data.name = vals[1];
+                    try
+                    {
+                        data.shape = ctx.ReadShapeFromWkt(vals[2]);
+                    }
+                    catch (ParseException e)
+                    {
+                        throw new ApplicationException(e.Message, e);
+                    }
+                    results.Add(data);
+                }
+            }
+            finally
+            {
+                bufInput.Dispose();
+            }
+            return results.GetEnumerator();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/SpatialTestQuery.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/SpatialTestQuery.cs b/src/Lucene.Net.Tests.Spatial/SpatialTestQuery.cs
new file mode 100644
index 0000000..1de5a92
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/SpatialTestQuery.cs
@@ -0,0 +1,102 @@
+\ufeffusing Lucene.Net.Spatial.Queries;
+using Lucene.Net.Support;
+using Spatial4n.Core.Context;
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+
+namespace Lucene.Net.Spatial
+{
+    /*
+     * 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.
+     */
+
+    /// <summary>
+    /// Helper class to execute queries
+    /// </summary>
+    public class SpatialTestQuery
+    {
+        public string testname;
+        public string line;
+        public int lineNumber = -1;
+        public SpatialArgs args;
+        public List<string> ids = new List<string>();
+
+        /**
+         * Get Test Queries.  The InputStream is closed.
+         */
+        public static IEnumerator<SpatialTestQuery> GetTestQueries(
+            SpatialArgsParser parser,
+            SpatialContext ctx,
+            string name,
+            Stream @in)
+        {
+
+            List<SpatialTestQuery> results = new List<SpatialTestQuery>();
+
+            TextReader bufInput = new StreamReader(@in, Encoding.UTF8);
+            try
+            {
+                String line;
+                for (int lineNumber = 1; (line = bufInput.ReadLine()) != null; lineNumber++)
+                {
+                    SpatialTestQuery test = new SpatialTestQuery();
+                    test.line = line;
+                    test.lineNumber = lineNumber;
+
+                    try
+                    {
+                        // skip a comment
+                        if (line.StartsWith("[", StringComparison.Ordinal))
+                        {
+                            int idx2 = line.IndexOf(']');
+                            if (idx2 > 0)
+                            {
+                                line = line.Substring(idx2 + 1);
+                            }
+                        }
+
+                        int idx = line.IndexOf('@');
+                        StringTokenizer st = new StringTokenizer(line.Substring(0, idx - 0));
+                        while (st.HasMoreTokens())
+                        {
+                            test.ids.Add(st.NextToken().Trim());
+                        }
+                        test.args = parser.Parse(line.Substring(idx + 1).Trim(), ctx);
+                        results.Add(test);
+                    }
+                    catch (Exception ex)
+                    {
+                        throw new ApplicationException("invalid query line: " + test.line, ex);
+                    }
+                }
+            }
+            finally
+            {
+                bufInput.Dispose();
+            }
+            return results.GetEnumerator();
+        }
+
+        public override String ToString()
+        {
+            if (line != null)
+                return line;
+            return args.toString() + " " + ids;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/StrategyTestCase.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/StrategyTestCase.cs b/src/Lucene.Net.Tests.Spatial/StrategyTestCase.cs
new file mode 100644
index 0000000..01c8e67
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/StrategyTestCase.cs
@@ -0,0 +1,274 @@
+\ufeffusing Lucene.Net.Documents;
+using Lucene.Net.Index;
+using Lucene.Net.Queries.Function;
+using Lucene.Net.Search;
+using Lucene.Net.Spatial.Queries;
+using Lucene.Net.Util;
+using Spatial4n.Core.Context;
+using Spatial4n.Core.Shapes;
+using System;
+using System.Collections.Generic;
+using System.IO;
+
+namespace Lucene.Net.Spatial
+{
+    /*
+     * 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.
+     */
+
+    public abstract class StrategyTestCase : SpatialTestCase
+    {
+        public const string RESOURCE_PATH = "Lucene.Net.Tests.Spatial.Test_Files.";
+        public const string DATA_RESOURCE_PATH = RESOURCE_PATH + "Data.";
+
+        public static readonly String DATA_SIMPLE_BBOX = "simple-bbox.txt";
+        public static readonly String DATA_STATES_POLY = "states-poly.txt";
+        public static readonly String DATA_STATES_BBOX = "states-bbox.txt";
+        public static readonly String DATA_COUNTRIES_POLY = "countries-poly.txt";
+        public static readonly String DATA_COUNTRIES_BBOX = "countries-bbox.txt";
+        public static readonly String DATA_WORLD_CITIES_POINTS = "world-cities-points.txt";
+
+        public static readonly String QTEST_States_IsWithin_BBox = "states-IsWithin-BBox.txt";
+        public static readonly String QTEST_States_Intersects_BBox = "states-Intersects-BBox.txt";
+        public static readonly String QTEST_Cities_Intersects_BBox = "cities-Intersects-BBox.txt";
+        public static readonly String QTEST_Simple_Queries_BBox = "simple-Queries-BBox.txt";
+
+        //private Logger log = Logger.getLogger(getClass().getName());
+
+        protected readonly SpatialArgsParser argsParser = new SpatialArgsParser();
+
+        protected SpatialStrategy strategy;
+        protected bool storeShape = true;
+
+        protected virtual void executeQueries(SpatialMatchConcern concern, params string[] testQueryFile)
+        {
+            //log.info("testing queried for strategy "+strategy);
+            foreach (String path in testQueryFile)
+            {
+                IEnumerator<SpatialTestQuery> testQueryIterator = getTestQueries(path, ctx);
+                runTestQueries(testQueryIterator, concern);
+            }
+        }
+
+        protected virtual void getAddAndVerifyIndexedDocuments(String testDataFile)
+        {
+            List<Document> testDocuments = getDocuments(testDataFile);
+            addDocumentsAndCommit(testDocuments);
+            VerifyDocumentsIndexed(testDocuments.size());
+        }
+
+        protected virtual List<Document> getDocuments(String testDataFile)
+        {
+            return getDocuments(getSampleData(testDataFile));
+        }
+
+        protected virtual List<Document> getDocuments(IEnumerator<SpatialTestData> sampleData)
+        {
+            List<Document> documents = new List<Document>();
+            while (sampleData.MoveNext())
+            {
+                SpatialTestData data = sampleData.Current;
+                Document document = new Document();
+                document.Add(new StringField("id", data.id, Field.Store.YES));
+                document.Add(new StringField("name", data.name, Field.Store.YES));
+                IShape shape = data.shape;
+                shape = convertShapeFromGetDocuments(shape);
+                if (shape != null)
+                {
+                    foreach (Field f in strategy.CreateIndexableFields(shape))
+                    {
+                        document.Add(f);
+                    }
+                    if (storeShape)//just for diagnostics
+                        document.Add(new StoredField(strategy.FieldName, shape.toString()));
+                }
+
+                documents.Add(document);
+            }
+            return documents;
+        }
+
+        /** Subclasses may override to transform or remove a shape for indexing */
+        protected virtual IShape convertShapeFromGetDocuments(IShape shape)
+        {
+            return shape;
+        }
+
+        protected virtual IEnumerator<SpatialTestData> getSampleData(String testDataFile)
+        {
+            String path = DATA_RESOURCE_PATH + testDataFile;
+            Stream stream = GetType().Assembly.GetManifestResourceStream(path);
+            if (stream == null)
+                throw new FileNotFoundException("classpath resource not found: " + path);
+            return SpatialTestData.GetTestData(stream, ctx);//closes the InputStream
+        }
+
+        protected virtual IEnumerator<SpatialTestQuery> getTestQueries(String testQueryFile, SpatialContext ctx)
+        {
+            Stream @in = GetType().Assembly.GetManifestResourceStream(RESOURCE_PATH + testQueryFile);
+            return SpatialTestQuery.GetTestQueries(
+                argsParser, ctx, testQueryFile, @in);//closes the InputStream
+        }
+
+        public virtual void runTestQueries(
+            IEnumerator<SpatialTestQuery> queries,
+            SpatialMatchConcern concern)
+        {
+            while (queries.MoveNext())
+            {
+                SpatialTestQuery q = queries.Current;
+                runTestQuery(concern, q);
+            }
+        }
+
+        public virtual void runTestQuery(SpatialMatchConcern concern, SpatialTestQuery q)
+        {
+            String msg = q.toString(); //"Query: " + q.args.toString(ctx);
+            SearchResults got = executeQuery(makeQuery(q), Math.Max(100, q.ids.size() + 1));
+            if (storeShape && got.numFound > 0)
+            {
+                //check stored value is there
+                assertNotNull(got.results[0].document.Get(strategy.FieldName));
+            }
+            if (concern.orderIsImportant)
+            {
+                IEnumerator<String> ids = q.ids.GetEnumerator();
+                foreach (SearchResult r in got.results)
+                {
+                    String id = r.document.Get("id");
+                    if (!ids.MoveNext())
+                    {
+                        fail(msg + " :: Did not get enough results.  Expect" + q.ids + ", got: " + got.toDebugString());
+                    }
+                    assertEquals("out of order: " + msg, ids.Current, id);
+                }
+
+                if (ids.MoveNext())
+                {
+                    fail(msg + " :: expect more results then we got: " + ids.Current);
+                }
+            }
+            else
+            {
+                // We are looking at how the results overlap
+                if (concern.resultsAreSuperset)
+                {
+                    ISet<string> found = new HashSet<string>();
+                    foreach (SearchResult r in got.results)
+                    {
+                        found.add(r.document.Get("id"));
+                    }
+                    foreach (String s in q.ids)
+                    {
+                        if (!found.contains(s))
+                        {
+                            fail("Results are mising id: " + s + " :: " + found);
+                        }
+                    }
+                }
+                else
+                {
+                    List<string> found = new List<string>();
+                    foreach (SearchResult r in got.results)
+                    {
+                        found.Add(r.document.Get("id"));
+                    }
+
+                    // sort both so that the order is not important
+                    CollectionUtil.TimSort(q.ids);
+                    CollectionUtil.TimSort(found);
+                    assertEquals(msg, q.ids.toString(), found.toString());
+                }
+            }
+        }
+
+        protected virtual Query makeQuery(SpatialTestQuery q)
+        {
+            return strategy.MakeQuery(q.args);
+        }
+
+        protected virtual void adoc(String id, String shapeStr)
+        {
+            IShape shape = shapeStr == null ? null : ctx.ReadShapeFromWkt(shapeStr);
+            AddDocument(newDoc(id, shape));
+        }
+        protected virtual void adoc(String id, IShape shape)
+        {
+            AddDocument(newDoc(id, shape));
+        }
+
+        protected virtual Document newDoc(String id, IShape shape)
+        {
+            Document doc = new Document();
+            doc.Add(new StringField("id", id, Field.Store.YES));
+            if (shape != null)
+            {
+                foreach (Field f in strategy.CreateIndexableFields(shape))
+                {
+                    doc.Add(f);
+                }
+                if (storeShape)
+                    doc.Add(new StoredField(strategy.FieldName, shape.toString()));//not to be parsed; just for debug
+            }
+            return doc;
+        }
+
+        protected virtual void DeleteDoc(String id)
+        {
+            indexWriter.DeleteDocuments(new TermQuery(new Term("id", id)));
+        }
+
+        /** scores[] are in docId order */
+        protected virtual void CheckValueSource(ValueSource vs, float[] scores, float delta)
+        {
+            FunctionQuery q = new FunctionQuery(vs);
+
+            //    //TODO is there any point to this check?
+            //    int expectedDocs[] = new int[scores.length];//fill with ascending 0....length-1
+            //    for (int i = 0; i < expectedDocs.length; i++) {
+            //      expectedDocs[i] = i;
+            //    }
+            //    CheckHits.checkHits(random(), q, "", indexSearcher, expectedDocs);
+
+            TopDocs docs = indexSearcher.Search(q, 1000);//calculates the score
+            for (int i = 0; i < docs.ScoreDocs.Length; i++)
+            {
+                ScoreDoc gotSD = docs.ScoreDocs[i];
+                float expectedScore = scores[gotSD.Doc];
+                assertEquals("Not equal for doc " + gotSD.Doc, expectedScore, gotSD.Score, delta);
+            }
+
+            CheckHits.CheckExplanations(q, "", indexSearcher);
+        }
+
+        protected virtual void AssertOperation(IDictionary<String, IShape> indexedDocs,
+                                       SpatialOperation operation, IShape queryShape)
+        {
+            //Generate truth via brute force
+            ISet<string> expectedIds = new HashSet<string>();
+            foreach (var stringShapeEntry in indexedDocs)
+            {
+                if (operation.Evaluate(stringShapeEntry.Value, queryShape))
+                    expectedIds.add(stringShapeEntry.Key);
+            }
+
+            SpatialTestQuery testQuery = new SpatialTestQuery();
+            testQuery.args = new SpatialArgs(operation, queryShape);
+            testQuery.ids = new List<string>(expectedIds);
+            runTestQuery(SpatialMatchConcern.FILTER, testQuery);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Test-Files/Data/LUCENE-4464.txt
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Test-Files/Data/LUCENE-4464.txt b/src/Lucene.Net.Tests.Spatial/Test-Files/Data/LUCENE-4464.txt
new file mode 100644
index 0000000..dfb5a40
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Test-Files/Data/LUCENE-4464.txt
@@ -0,0 +1,3 @@
+#id	name	shape
+poly1	poly1	POLYGON ((-93.17288720912401 45.280265431486754, -93.17232270645628 45.2802724629027, -93.17229737711205 45.279497574052314, -93.1722224854913 45.277577770983854, -93.17218124644266 45.276747010395624, -93.16722650828461 45.276819421108826, -93.16581262076448 45.27684404529939, -93.16363038333625 45.276882054199596, -93.16249244695301 45.276929493877525, -93.16247370542268 45.27641118002343, -93.16246893668628 45.276279382682894, -93.1624671302382 45.274701063846244, -93.16246679905096 45.273381422360785, -93.16247689122851 45.273189685068424, -93.16249146710186 45.27291249464421, -93.16249868565903 45.272467966062614, -93.16247955957382 45.27177209534391, -93.1624787718002 45.27127651548793, -93.16247840794293 45.27104491547271, -93.16247917486976 45.27087000356473, -93.1624817727418 45.270279315147775, -93.16252487154968 45.26996729342093, -93.16254025661699 45.26976826077157, -93.16247902564132 45.269527941604, -93.16242684845764 45.2692774997531, -93.16242126018722 4
 5.26894470083864, -93.16241263011544 45.26769394309626, -93.16246809168283 45.26571736107859, -93.16247263940593 45.26195548919013, -93.16253090997651 45.258615729449964, -93.16256878834184 45.25650987969364, -93.1626048203569 45.2546538608912, -93.16265873943591 45.251876274357876, -93.16275002007988 45.2510418534315, -93.16282237443883 45.25042383853711, -93.16286421513767 45.249181538840595, -93.16288289220509 45.24862697953288, -93.1629601120395 45.248250613185206, -93.16301002807151 45.24802483983211, -93.16301621932013 45.247670020958665, -93.16301519349018 45.247478630666144, -93.16303001333274 45.24727504082362, -93.16303463142393 45.24713931946277, -93.16302280990728 45.2470107542477, -93.16298327344437 45.24685970499298, -93.16294217154733 45.246633449219054, -93.16294315088282 45.246419514713516, -93.16295754265565 45.24621538933992, -93.16296755618336 45.24580786412655, -93.16296268372803 45.245362220836384, -93.16296319568123 45.245046689033444, -93.16297766811293 45.24
 481357093532, -93.16296370759883 45.2445699039253, -93.16294931051515 45.24231310924752, -93.16294559876471 45.24173111255096, -93.16295568091667 45.240776604513705, -93.1629609359182 45.24053954238007, -93.1629658719288 45.24019639978025, -93.1625355179785 45.24018482062359, -93.15847246037083 45.24007549519542, -93.15641780558727 45.24006372373029, -93.15470331938288 45.24002991133718, -93.1515176880772 45.240038275846665, -93.14892151971884 45.24004508944476, -93.14597353408716 45.240012024375574, -93.14198169289922 45.239944427606616, -93.14246140322608 45.21441838866706, -93.14239730934507 45.20842345035032, -93.14240307538512 45.203669567890245, -93.13209436867183 45.20385828388066, -93.13238731320574 45.19696183064252, -93.13244550539693 45.19559178376392, -93.13255875219626 45.19292582294682, -93.12747185962866 45.19303831675316, -93.12741613255534 45.196689407842044, -93.12341724811418 45.196748516850086, -93.12336451543653 45.19630050937325, -93.12233270487748 45.196311891
 79194, -93.12244695905335 45.18943470505876, -93.12752867296823 45.18931969757398, -93.1275981937757 45.18579899512077, -93.12249095182051 45.18589579364393, -93.12250905286206 45.18230218633591, -93.11745336177542 45.182234528897865, -93.11742994994425 45.17494109686777, -93.11234677240823 45.174914625057596, -93.11232755368056 45.178541858988, -93.09142510557425 45.17830768889981, -93.0878908215621 45.18208021181682, -93.04087986544745 45.182020129318005, -93.02011304608662 45.18206919600553, -92.99725469269949 45.18154883703301, -92.9866455346556 45.18162938363265, -92.98002761377205 45.181741313792635, -92.97460481311676 45.1817232745721, -92.95815138711436 45.18159971137449, -92.95832448011389 45.16710586357575, -92.95821211351648 45.15266682925307, -92.94804883291458 45.152678829402525, -92.94820512323935 45.14582287000843, -92.94821449767262 45.14541149629351, -92.93808126859899 45.145435393255234, -92.938064080176 45.1464755574292, -92.93790172782569 45.15630033200825, -92.9
 3776855788026 45.156299483202375, -92.93416458772786 45.15627656196406, -92.92776593175911 45.156235863288074, -92.92779198321185 45.15260820059608, -92.9228643837518 45.15257871636257, -92.91761510291013 45.15254730117589, -92.91755895303478 45.15978011255037, -92.90742527225278 45.15975884768774, -92.90734951861361 45.16700513027527, -92.90243435593408 45.16697925148226, -92.90226994175299 45.16697838648701, -92.90228225598396 45.16960751885433, -92.90228682505473 45.170583562524534, -92.89838293958822 45.17058359192683, -92.89776337384279 45.17058359923907, -92.89720228241329 45.170636798053465, -92.89720546113311 45.171648743169875, -92.89721045187194 45.17323675651512, -92.89721215942521 45.17377958217219, -92.8972133713998 45.17416655315385, -92.89752994284902 45.17416793500262, -92.90230392627396 45.174188700362095, -92.90230695467396 45.17483317173849, -92.90230939234701 45.175352265892315, -92.90231342163983 45.17620891826606, -92.9023378718661 45.18141217320357, -92.898291
 95794003 45.18137903577816, -92.89197067471983 45.181327269964534, -92.86573042754982 45.18111238484799, -92.86537258386163 45.18110945889712, -92.86579788828743 45.16683341076013, -92.85850341291456 45.166840495697045, -92.85576616527777 45.1668651317465, -92.8455814929548 45.16695680639518, -92.8403672382906 45.167003741522834, -92.84037534438275 45.166359277271084, -92.83914257524022 45.166407761467035, -92.83786182101709 45.16655768366541, -92.83762301824869 45.16658563659705, -92.83700510809494 45.16665797101126, -92.83700330475195 45.1670405349812, -92.83520392476423 45.16704646605868, -92.83519998302931 45.1672093811339, -92.83518241658018 45.17114095264113, -92.8351705215998 45.17380185475555, -92.83516823773242 45.17431412368648, -92.82501384033566 45.174380025018145, -92.82373302900695 45.174963166130034, -92.82127603798283 45.17799740439804, -92.81495695139105 45.17798284134312, -92.81498212776123 45.18394380043827, -92.81496335262872 45.20297631525698, -92.81496300732859
  45.2033351264244, -92.8149190887153 45.20460132029917, -92.81473397710002 45.21, -92.8198460035041 45.21, -92.81985864578533 45.21352006541341, -92.81476009958381 45.21350519453624, -92.81473397710002 45.211, -92.79434616877515 45.20979982288059, -92.79434485197183 45.210003888814526, -92.7942994128934 45.217028016258524, -92.79414754531777 45.217027433538036, -92.75950558164095 45.216895251116746, -92.75791266717471 45.216889175072694, -92.75634408090858 45.21737192056616, -92.75539334998972 45.21781096867505, -92.75544275719047 45.219840930849315, -92.75232263931744 45.219847708152834, -92.75345360864661 45.22241622713623, -92.75393100188802 45.22290500013628, -92.75454911801587 45.22425238152991, -92.75465656863904 45.22441872007679, -92.75478824580995 45.22461252606749, -92.75573200183275 45.22594899943625, -92.7559326169467 45.2263989667922, -92.756173357985 45.22677479396459, -92.75628338889855 45.227185737281864, -92.75651400327136 45.22770300256764, -92.75667800355963 45.22
 8069998932774, -92.75745600158125 45.23052599674398, -92.75737071502948 45.23131853178694, -92.75760683805547 45.23212889115611, -92.7575248338702 45.23249816977935, -92.75760900807862 45.233043995948975, -92.75740715667484 45.23498808590038, -92.75739258433605 45.23515457917446, -92.75736004212973 45.235441823970014, -92.75728900664646 45.2361259970008, -92.75750924881613 45.23833187652166, -92.75783421241928 45.239151014730965, -92.75799784052033 45.2401986059374, -92.75814399470411 45.24075700093086, -92.75910499448543 45.24444199845027, -92.75927217262658 45.246363482652335, -92.759708376526 45.24795052230262, -92.76024900009054 45.24960000150479, -92.76026400206055 45.25171699829065, -92.75984499770836 45.25286799832034, -92.75883599655404 45.25442699925451, -92.75592228367496 45.256779108256175, -92.75559993467031 45.25707105760005, -92.75540261715516 45.25725539605134, -92.75458100472993 45.258140999051975, -92.75362100152239 45.25941899619891, -92.75258800661327 45.261786002
 1943, -92.7523530053651 45.26244399793552, -92.7521330910868 45.26318539548715, -92.75199986320791 45.26381589028983, -92.7519440909167 45.26415703570502, -92.75192391851121 45.26559725594415, -92.75247612752318 45.26746623235666, -92.75254008932185 45.26768063816608, -92.75267394173396 45.268130176728555, -92.75287910082022 45.2688320393691, -92.7530104867237 45.26921012533672, -92.75329204456183 45.26980089141646, -92.75414711285153 45.2712720735891, -92.7552129966957 45.27237299947564, -92.75574299378961 45.27288399662051, -92.75678399520334 45.273891998902435, -92.75750199664172 45.27442999825494, -92.75801999923948 45.274822998224586, -92.75866321741752 45.27578539520815, -92.7589271849383 45.27616491445647, -92.75924599787822 45.27671899844492, -92.75941999802778 45.27718649803985, -92.75960999785612 45.27731999914, -92.75978699565532 45.27743849638546, -92.76004300142414 45.277978995119405, -92.76061199738588 45.27882799808139, -92.76117799722955 45.280582999200305, -92.76136
 19999475 45.28220800042353, -92.76167096088638 45.2836803717185, -92.76198517744629 45.2850267976271, -92.76206945308458 45.2853507773657, -92.76202745146396 45.286658659028, -92.76204199858486 45.28698499388888, -92.76201199644161 45.28793199672008, -92.76200399722086 45.28821299803955, -92.76121399640145 45.28913599914764, -92.7603870028136 45.28991599406784, -92.75871000510011 45.29096499709372, -92.75799200634881 45.291140996050984, -92.75687800551285 45.29148399845183, -92.75507700319366 45.2919269952758, -92.75480030147037 45.291986779669465, -92.74569331443023 45.29606484000191, -92.74555580404507 45.29614422445335, -92.74523588498667 45.29631411941847, -92.76071968429389 45.29617634034589, -92.79448651640953 45.29587194744184, -92.82553071142016 45.29634288822895, -92.82523623967 45.28697641600944, -92.8246113114385 45.27459391718561, -92.82414631698042 45.26733414102221, -92.83443181636859 45.267466042102846, -92.83450366471794 45.265666722695805, -92.8395297613521 45.26570
 782145342, -92.83954651660255 45.2675117790906, -92.85488466565545 45.267633226883305, -92.85446380439222 45.260381978642265, -92.8530801070886 45.256940031152055, -92.8746167542768 45.2569553750289, -92.87517983690772 45.26774272327855, -92.88032459143679 45.26775272915376, -92.88028907325248 45.27498539130476, -92.885429695981 45.27499516876503, -92.88541044770409 45.27862274921294, -92.8854460740016 45.28269595676258, -92.8858306795285 45.28583335680999, -92.89095994168375 45.285838365551086, -92.89147668909354 45.290056047991875, -92.89183494474656 45.292995365557246, -92.89287941280966 45.29621886928581, -92.93574219102997 45.296382695230655, -92.9366855829562 45.29639453639271, -92.93730010601949 45.29640233268984, -92.93773633826109 45.296407862218295, -92.95031707870098 45.29656663627082, -92.95732733387652 45.29663267857854, -92.95723233585932 45.305785498930874, -92.95755812361517 45.31807293816823, -92.9575313307762 45.325662873647204, -92.96200814151011 45.32569410734573
 , -92.96201051236334 45.33056403262943, -92.95763365021791 45.330562956294486, -92.95750484414667 45.34006528297348, -92.95740249422305 45.3523406680097, -92.96272753035339 45.352295608902175, -92.96260253143201 45.363259386181184, -92.95732537061275 45.363286992831206, -92.95715614538045 45.36869421119079, -92.97302216756823 45.36904156334545, -92.9731090974606 45.37554810693529, -92.98760985309234 45.37555619312347, -92.98429494637762 45.38215591061988, -92.9924184629002 45.38233326055907, -93.01850137881846 45.38277378724873, -93.01956464133914 45.41174708503911, -93.03973263863047 45.412106304897264, -93.06569776540464 45.412656360563524, -93.08346874844985 45.41297273973574, -93.09263091377308 45.41335460313747, -93.1012213163472 45.413720365424695, -93.10759754754753 45.41373499082408, -93.14214551761233 45.41373101611429, -93.1421802894172 45.40666589187203, -93.14209155741717 45.38498980813781, -93.14398965535287 45.369981475770224, -93.13861914028635 45.36992203894643, -93.
 13946982733188 45.35540022959687, -93.14362673736643 45.35542059147377, -93.14338145836778 45.34816201728363, -93.14259222919002 45.34815677471413, -93.14123737100095 45.34271091215897, -93.14120170425102 45.34166175650565, -93.14159640367895 45.340845226624126, -93.16430988689314 45.34107128935172, -93.1641229508536 45.33731028186903, -93.163783504365 45.32713863170596, -93.16354815472778 45.31568179036097, -93.1634974864936 45.3115083559682, -93.16335415000293 45.30838048844207, -93.16326942872365 45.30653168298998, -93.16286993093225 45.29781375116957, -93.16292479029 45.297483756012355, -93.16251838572086 45.29748043583636, -93.16242411934059 45.29340169752503, -93.16237192435095 45.291513658346155, -93.16125915756838 45.29101148729498, -93.16224903398384 45.290456018307964, -93.16243543883762 45.29031474509565, -93.16248365754952 45.29016960982244, -93.1625270557542 45.28932067928762, -93.16350507037129 45.28940282906675, -93.16413761242012 45.28944739938537, -93.16430369461645
  45.289411531953206, -93.164472138656 45.28937514511818, -93.16431016328954 45.288334379584406, -93.16422830296436 45.28780835028316, -93.16373011428878 45.287807744950875, -93.16348868413621 45.28778563548775, -93.16304669211718 45.28779811404454, -93.16252493722239 45.28781182501286, -93.1625182014603 45.28601279964026, -93.1625127377889 45.28416325442296, -93.1717122152211 45.28391079701647, -93.17291828928865 45.28387769615237, -93.17292468588315 45.28327561174209, -93.1729215958459 45.28269914269899, -93.17290904354249 45.28216703245599, -93.17290447076888 45.281410092382885, -93.17289432485279 45.28068732375472, -93.17288720912401 45.280265431486754))
+poly2	poly2	POLYGON((-93.26592485308495 45.18931973506328, -93.26373519655886 45.18933815615675, -93.2637828223868 45.18660121752107, -93.26280973893772 45.18656958194617, -93.2603275028686 45.186488876325654, -93.25976682936536 45.18646929139094, -93.25877703935303 45.18686109057519, -93.25788401039608 45.18633824889261, -93.25713811973642 45.186864792015704, -93.25660115549654 45.18628640445176, -93.24081325108644 45.18609354693712, -93.2356823133177 45.1860308697061, -93.23474944979115 45.186019474019865, -93.23478565684188 45.18266103967549, -93.23072066106351 45.18267669158043, -93.22480340476464 45.18267437402639, -93.21952101307244 45.18267371221728, -93.21950131879755 45.184689058075534, -93.21950381582634 45.18590104693386, -93.21950547892035 45.186708829298695, -93.21948324866376 45.18808573281868, -93.21947477056304 45.188619717930756, -93.2194751507154 45.1899146284615, -93.22390334137022 45.18991091026497, -93.2245904557543 45.18993775453468, -93.2245784309098 45.190287
 02856576, -93.2245932424241 45.19081834295508, -93.22460314163764 45.19137779927979, -93.22459067695124 45.19162607300785, -93.22458367100289 45.19176562022696, -93.22354968949122 45.191760188521705, -93.22131530006368 45.19175468785821, -93.22018302807493 45.19175762419069, -93.21965635944291 45.19175898704962, -93.21824735047468 45.191762639857636, -93.21840068968908 45.191840907619024, -93.21858279007587 45.191950538176606, -93.21874378970492 45.19205449060312, -93.21893581214327 45.192204972059955, -93.21911499957261 45.19238205879934, -93.21934767139433 45.192628269473076, -93.21954522989743 45.1928508489684, -93.21972003978802 45.19304459976245, -93.21997538064213 45.19332124206717, -93.22011354045264 45.193470928079385, -93.22046875034326 45.19384479955501, -93.2206469058326 45.19404172922978, -93.22079845082156 45.194244494502364, -93.2209416400795 45.19447508772328, -93.22107397875365 45.19474417974581, -93.2211368505518 45.19490985928749, -93.22118231976518 45.195047277731
 625, -93.22124659963487 45.19525315038074, -93.22128314962913 45.195396480693944, -93.22130715028514 45.195564823375, -93.22131862069979 45.195757013030224, -93.22130704484326 45.19599065847414, -93.22127083850016 45.19622942989826, -93.22124456959293 45.19636257994296, -93.22120917947201 45.19651471803614, -93.22115328972328 45.196774039833144, -93.22110053150747 45.19700410181286, -93.22105123806169 45.19721904984113, -93.21939747849284 45.19720754776318, -93.21658707902952 45.19719901749774, -93.21405492494755 45.19718389708806, -93.21060961905127 45.19716332241369, -93.20846870851273 45.19715738191871, -93.20635420918421 45.19714993030806, -93.20384995444252 45.19713947337882, -93.20382099935851 45.195915480832355, -93.20379040854755 45.195493880093856, -93.20373937951182 45.19525460196455, -93.20366799901262 45.194730001052676, -93.20359944927 45.194273469702246, -93.20351980946141 45.19386975065817, -93.20336890147132 45.1933312322322, -93.20348773988103 45.19317805926476, -93
 .20364964522179 45.19294381603321, -93.20373782170354 45.192758795441485, -93.20378634041538 45.1925589245846, -93.20378780054193 45.1924118820702, -93.20373224993294 45.192246366644895, -93.20366678053941 45.192063182244134, -93.20349712021084 45.19164111034226, -93.20336402335359 45.191262445660406, -93.20333661484061 45.19107258136713, -93.20334012614478 45.19082850506992, -93.20338500114326 45.190584969374704, -93.20346313590359 45.19035226093307, -93.20353125074365 45.19015096025676, -93.20337886118753 45.19012069933683, -93.20280004152556 45.18999823901699, -93.20236430223584 45.1898748712581, -93.20223796285948 45.18983446401002, -93.20171338128353 45.189666689690526, -93.20105175026708 45.18940210042135, -93.20059509118217 45.18937347081525, -93.20014399997638 45.18935951962055, -93.1999096512546 45.18934032171285, -93.19969162075753 45.18934030912719, -93.19953079227915 45.18938062079311, -93.19930724128803 45.189471810355066, -93.19836742091539 45.18954495845859, -93.19790
 904174889 45.189755310346555, -93.19770094626355 45.18978905045578, -93.19728573057267 45.1898563687543, -93.19706717806918 45.18978234280038, -93.1961191012612 45.18980511056629, -93.19583707702907 45.18977039110604, -93.19495714548943 45.18966207098092, -93.19409949054268 45.18955648989894, -93.19361391124465 45.18954758129998, -93.19142135137997 45.189507349701145, -93.18867729058191 45.18943758222878, -93.18766468614145 45.18941183701645, -93.1869063815807 45.18939255950494, -93.18676117212036 45.18939312363656, -93.18583601993124 45.18939673056086, -93.18362870083628 45.18940533739182, -93.18015920861117 45.189432919714875, -93.17748344774633 45.18940274982507, -93.17100678798263 45.18934067185518, -93.1680509570817 45.18931686702863, -93.16712265967519 45.189309389152754, -93.1632729184803 45.189289560128074, -93.1524420382428 45.189137301470666, -93.1488330300988 45.189087681208825, -93.14258337454692 45.18900953614207, -93.1425728385595 45.18964797148711, -93.14257129908563 
 45.19044710129245, -93.14256839076815 45.191380659844974, -93.14257549009486 45.192639988690985, -93.14256591028126 45.193624481846925, -93.1425562203409 45.19475816134898, -93.14254671019609 45.19564806883362, -93.14253591314012 45.19592629600891, -93.1425191002932 45.19635953895129, -93.14240307328147 45.20366956427245, -93.14239731024965 45.20842345007226, -93.14246141142196 45.2144183909345, -93.14198170032972 45.23994442974387, -93.14597353942523 45.240012030562795, -93.14892151981124 45.24004509174428, -93.15151768504401 45.24003827478177, -93.15470331907811 45.2400299112851, -93.15641781022819 45.240063720104146, -93.15847245794774 45.24007548756677, -93.16253551804624 45.24018481776239, -93.16296586932476 45.24019639699945, -93.16296093749654 45.240539543608094, -93.16295567833508 45.24077659970959, -93.16294559992268 45.24173110984731, -93.16294931429802 45.242313107885224, -93.16296371061823 45.24456989801016, -93.16297766989932 45.24481356907269, -93.16296319587042 45.245
 04668430867, -93.16296267909655 45.24536222031531, -93.16296756070733 45.24580785775435, -93.16295754084666 45.24621538734816, -93.16294315030365 45.24641950970948, -93.1629421699368 45.246633444731216, -93.16298326866249 45.24685970478054, -93.16302280494743 45.24701074802872, -93.1630346343297 45.247139320093076, -93.16303000914128 45.24727503858908, -93.16301519072017 45.24747862874394, -93.16301622062082 45.247670019373224, -93.16301002844395 45.24802483903903, -93.16296010836595 45.248250609285236, -93.16288288941641 45.248626979189, -93.16286421036493 45.24918153632857, -93.16282236866641 45.25042383853131, -93.16275001793326 45.25104184745623, -93.16265874011768 45.251876269431015, -93.1626048141941 45.25465385517585, -93.162568780952 45.25650987775294, -93.16253090903855 45.25861572819838, -93.16247264162719 45.261955487720506, -93.16246809047925 45.26571735738526, -93.16241263022145 45.267693939529536, -93.16242125944353 45.26894469986081, -93.16242684956876 45.269277499432
 015, -93.16247902269161 45.26952793567272, -93.16254025984375 45.269768259020054, -93.1625248689828 45.26996728874923, -93.16248176954191 45.27027930739088, -93.16247917649272 45.270869996810376, -93.16247840915516 45.27104490906511, -93.16247877426206 45.27127651283899, -93.162479560911 45.27177208702322, -93.16249869026827 45.272467959171365, -93.16249147172434 45.27291248854739, -93.16247688682598 45.27318968296259, -93.16246680083795 45.27338141702519, -93.1624671298516 45.27470105775956, -93.16246893968787 45.276279379505084, -93.1624737063593 45.2764111771935, -93.16249244905424 45.276929488819604, -93.16363037995181 45.27688204948932, -93.16581262202895 45.276844043452684, -93.16722651010657 45.27681941864911, -93.17218124072862 45.27674700948904, -93.1722224784459 45.27757776899891, -93.17229737034532 45.279497570305445, -93.17232269933695 45.28027246109518, -93.17288721010608 45.28026543129147, -93.1728943187817 45.2806873180744, -93.17290447218495 45.28141008817547, -93.17
 290904002667 45.28216703008146, -93.17292159084371 45.28269913830247, -93.17292468118433 45.283275608616165, -93.17291828224536 45.28387769767021, -93.1717122127579 45.283910797244246, -93.16251273143365 45.28416325629099, -93.16251820094257 45.28601279797615, -93.16252493935717 45.287811833132764, -93.16304669905364 45.28779811692505, -93.16348868871324 45.28778563925035, -93.16373011962693 45.28780774767522, -93.16422830587629 45.28780835110865, -93.1643101699488 45.28833437868018, -93.16447213914093 45.289375147768524, -93.16430369361024 45.28941153310711, -93.16413761723706 45.28944740219967, -93.16350507286433 45.289402832527344, -93.16252705964098 45.289320683284735, -93.16248365939401 45.29016961156254, -93.16243543831087 45.29031475002342, -93.16224903970826 45.2904560215217, -93.16125915934788 45.29101149209126, -93.16237192796683 45.291513661220456, -93.16242412151107 45.29340170072084, -93.16251838980172 45.29748044313293, -93.16292479370829 45.29748376064082, -93.1863909
 4534673 45.29767533425263, -93.18833342032521 45.29769119188229, -93.1925428426471 45.29770437859642, -93.19474753040078 45.29771128804242, -93.19765740975974 45.29769541872667, -93.20297591868295 45.29776263827187, -93.20683144906876 45.29774197003572, -93.20883497923562 45.297766559466794, -93.21546742887979 45.297768422222155, -93.22617724980643 45.29791971794424, -93.23408017640227 45.298023690859175, -93.2343080073169 45.288444186545625, -93.23432525195352 45.287995322205425, -93.23469515647318 45.269279712377234, -93.23475627635968 45.266203358381446, -93.23560542207227 45.26619551047824, -93.23899176558338 45.26613779367068, -93.24250527367546 45.26608234822973, -93.243445378056 45.26606503829342, -93.24512861083372 45.2660344570852, -93.24588057830995 45.26602026067889, -93.24713274287363 45.26599455787498, -93.25036838013868 45.26592734514467, -93.25172461510564 45.265900698298395, -93.25236738024864 45.265888260809106, -93.25481754173921 45.26583307838667, -93.255713579529
 06 45.265819559899164, -93.2594981489083 45.26575415212897, -93.26098138766197 45.265754375486374, -93.26155216698102 45.26565612540643, -93.26170097145753 45.26562288963898, -93.26208574477789 45.26553876835043, -93.26245875524685 45.265434673708015, -93.26277275191426 45.265316250819595, -93.26311663127117 45.26517251314189, -93.26346212923646 45.26500240317637, -93.26393572774133 45.26477558787491, -93.2651820516718 45.26406759657772, -93.26518110226205 45.26337226279194, -93.26515218908767 45.26311636791454, -93.26518703008779 45.262871689663605, -93.2652064900752 45.26265582104258, -93.2652110298225 45.26215614194132, -93.26522443086994 45.26112430402238, -93.26522989950563 45.260703199933474, -93.26524872191168 45.25930812973533, -93.26525187087448 45.258897852775995, -93.26525857049303 45.258025812056765, -93.26527734826267 45.256675072153314, -93.26528081766433 45.25612813038996, -93.265287399575 45.25512698071874, -93.26530031054412 45.253711671615115, -93.26531490547187 45
 .25273002640574, -93.26532214123614 45.252243491267, -93.26533817105908 45.25062180123498, -93.26535413994274 45.24906421173263, -93.26536141910549 45.24841165046578, -93.26536638602661 45.24796649509243, -93.26537318826473 45.24735637067748, -93.26539798003012 45.24589779189643, -93.265404909549 45.24454674190931, -93.2654060939449 45.24296904311022, -93.26540624905046 45.24276127146885, -93.26540843815205 45.2420263885843, -93.26541275006169 45.240577352345994, -93.2654375717671 45.238843301612725, -93.26544518264211 45.237906888690105, -93.26544940933664 45.23738688110566, -93.26546966016808 45.236093591927926, -93.2654781584622 45.235359229961944, -93.26548338867605 45.23490715107922, -93.26553582901259 45.23354268990693, -93.26554071996831 45.23330119833777, -93.26555987026248 45.2323552839169, -93.26557251955711 45.23173040973764, -93.26556626032777 45.22975235185782, -93.26556606661761 45.229367333607186, -93.26556579189545 45.228823722705066, -93.26562882232702 45.2268722061
 76665, -93.26571073971922 45.224335971082276, -93.26574560622672 45.22192222321787, -93.26574836877063 45.22173093256304, -93.26577033227747 45.22021043432355, -93.26578588443306 45.21913391123174, -93.26580662128347 45.21769799745153, -93.26580983179628 45.217475736026664, -93.26581322607608 45.217240685631346, -93.26590715360736 45.210737684073244, -93.26591966090616 45.209871711997586, -93.2659016992406 45.20722015227932, -93.26587484243684 45.203254836571126, -93.26585637174348 45.20052765082941, -93.26585684827346 45.19841676076085, -93.26587786763154 45.19732741144391, -93.2658624676632 45.1970879109074, -93.2659274100303 45.194004979577755, -93.26595017983325 45.191531890895845, -93.26595423366354 45.19092534610275, -93.26593099287571 45.190637988686554, -93.2659274057232 45.18986823069059, -93.26592485308495 45.18931973506328))
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d8c73530/src/Lucene.Net.Tests.Spatial/Test-Files/Data/countries-bbox.txt
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Test-Files/Data/countries-bbox.txt b/src/Lucene.Net.Tests.Spatial/Test-Files/Data/countries-bbox.txt
new file mode 100644
index 0000000..78e14e0
--- /dev/null
+++ b/src/Lucene.Net.Tests.Spatial/Test-Files/Data/countries-bbox.txt
@@ -0,0 +1,249 @@
+#id	name	shape	
+FLK	Falkland Is.	ENVELOPE(-61.148055, -57.733200, -51.249455, -52.343055)
+GUF	French Guiana	ENVELOPE(-54.603782, -51.648055, 5.755418, 2.113473)
+GUY	Guyana	ENVELOPE(-61.389727, -56.470636, 8.535273, 1.186873)
+PCN	Pitcairn Is.	ENVELOPE(-130.105055, -128.286118, -24.325836, -25.082227)
+SGS	South Georgia & the South Sandwich Is.	ENVELOPE(-38.023755, -26.241391, -53.989727, -58.498609)
+SHN	St. Helena	ENVELOPE(-5.792782, -5.645282, -15.903755, -16.021946)
+SUR	Suriname	ENVELOPE(-58.071400, -53.986118, 6.001809, 1.836245)
+TTO	Trinidad & Tobago	ENVELOPE(-61.921600, -60.520836, 11.345554, 10.040345)
+VEN	Venezuela	ENVELOPE(-73.378064, -59.803055, 12.197500, 0.649164)
+ASM	American Samoa	ENVELOPE(-170.823227, -170.561873, -14.254309, -14.375555)
+COK	Cook Is.	ENVELOPE(-165.848345, -157.703764, -10.881318, -21.940836)
+PYF	French Polynesia	ENVELOPE(-151.497773, -138.809755, -8.778191, -17.870836)
+UMI	Jarvis I.	ENVELOPE(-160.045164, -160.009464, -0.374309, -0.398055)
+NIU	Niue	ENVELOPE(-169.952236, -169.781555, -18.963336, -19.145555)
+WSM	Samoa	ENVELOPE(-172.780027, -171.429200, -13.460555, -14.057500)
+TKL	Tokelau	ENVELOPE(-171.862718, -171.843764, -9.170627, -9.218891)
+TON	Tonga	ENVELOPE(-175.360000, -173.906827, -18.568055, -21.268064)
+WLF	Wallis & Futuna	ENVELOPE(-178.190273, -176.121936, -13.214864, -14.323891)
+ARG	Argentina	ENVELOPE(-73.582300, -53.650009, -21.780518, -55.051673)
+BOL	Bolivia	ENVELOPE(-69.656191, -57.521118, -9.679191, -22.901109)
+BRA	Brazil	ENVELOPE(-74.004591, -34.792918, 5.272709, -33.741118)
+CHL	Chile	ENVELOPE(-109.446109, -66.420627, -17.505282, -55.902227)
+ECU	Ecuador	ENVELOPE(-91.663891, -75.216846, 1.437782, -5.000309)
+PRY	Paraguay	ENVELOPE(-62.643773, -54.243900, -19.296809, -27.584727)
+PER	Peru	ENVELOPE(-81.355146, -68.673909, -0.036873, -18.348546)
+URY	Uruguay	ENVELOPE(-58.438609, -53.098300, -30.096673, -34.943818)
+UMI	Baker I.	ENVELOPE(-176.467655, -176.455855, 0.222573, 0.215282)
+CAN	Canada	ENVELOPE(-141.002991, -52.617364, 83.113873, 41.675554)
+GTM	Guatemala	ENVELOPE(-92.246782, -88.214736, 17.821109, 13.745836)
+UMI	Howland I.	ENVELOPE(-176.643082, -176.631091, 0.808609, 0.790282)
+UMI	Johnston Atoll	ENVELOPE(-169.538936, -169.523927, 16.730273, 16.724164)
+MEX	Mexico	ENVELOPE(-118.404164, -86.738618, 32.718454, 14.550545)
+UMI	Midway Is.	ENVELOPE(-177.395845, -177.360545, 28.221518, 28.184154)
+BRB	Barbados	ENVELOPE(-59.659446, -59.427082, 13.337082, 13.050554)
+DMA	Dominica	ENVELOPE(-61.491391, -61.250700, 15.631945, 15.198054)
+GRD	Grenada	ENVELOPE(-61.785182, -61.596391, 12.237154, 11.996945)
+GLP	Guadeloupe	ENVELOPE(-61.796109, -61.187082, 16.512918, 15.870000)
+MTQ	Martinique	ENVELOPE(-61.231536, -60.816946, 14.880136, 14.402773)
+LCA	St. Lucia	ENVELOPE(-61.079582, -60.878064, 14.109309, 13.709445)
+SPM	St. Pierre & Miquelon	ENVELOPE(-56.397782, -56.145500, 47.135827, 46.747191)
+VCT	St. Vincent & the Grenadines	ENVELOPE(-61.280146, -61.120282, 13.383191, 13.130282)
+ABW	Aruba	ENVELOPE(-70.059664, -69.874864, 12.627773, 12.411109)
+BMU	Bermuda	ENVELOPE(-64.823064, -64.676809, 32.379509, 32.260554)
+DOM	Dominican Republic	ENVELOPE(-72.003064, -68.322927, 19.930827, 17.604164)
+HTI	Haiti	ENVELOPE(-74.467791, -71.629182, 20.091454, 18.022782)
+JAM	Jamaica	ENVELOPE(-78.373900, -76.221118, 18.522500, 17.697218)
+ANT	Netherlands Antilles	ENVELOPE(-69.163618, -68.192927, 12.383891, 12.020554)
+BHS	The Bahamas	ENVELOPE(-78.978900, -72.738891, 26.929164, 20.915273)
+TCA	Turks & Caicos Is.	ENVELOPE(-72.031464, -71.127573, 21.957773, 21.429918)
+BLZ	Belize	ENVELOPE(-89.216400, -87.779591, 18.489900, 15.889854)
+CYM	Cayman Is.	ENVELOPE(-81.400836, -81.093064, 19.354164, 19.265000)
+COL	Colombia	ENVELOPE(-81.720146, -66.870455, 12.590273, -4.236873)
+CRI	Costa Rica	ENVELOPE(-85.911391, -82.561400, 11.212845, 8.025673)
+CUB	Cuba	ENVELOPE(-84.952927, -74.131255, 23.194027, 19.821945)
+SLV	El Salvador	ENVELOPE(-90.108064, -87.694673, 14.431982, 13.156391)
+HND	Honduras	ENVELOPE(-89.350491, -83.131855, 16.435827, 12.985173)
+NIC	Nicaragua	ENVELOPE(-87.689827, -83.131855, 15.022218, 10.709691)
+PAN	Panama	ENVELOPE(-83.030291, -77.198336, 9.620136, 7.206109)
+AIA	Anguilla	ENVELOPE(-63.167782, -62.972709, 18.272982, 18.164445)
+ATG	Antigua & Barbuda	ENVELOPE(-61.891109, -61.666946, 17.724300, 16.989718)
+VGB	British Virgin Is.	ENVELOPE(-64.698482, -64.324527, 18.504854, 18.383891)
+MSR	Montserrat	ENVELOPE(-62.236946, -62.138891, 16.812354, 16.671391)
+PRI	Puerto Rico	ENVELOPE(-67.266400, -65.301118, 18.519445, 17.922218)
+KNA	St. Kitts & Nevis	ENVELOPE(-62.862782, -62.622509, 17.410136, 17.208882)
+VIR	Virgin Is.	ENVELOPE(-65.023509, -64.562573, 18.387673, 17.676664)
+FRO	Faroe Is.	ENVELOPE(-7.433473, -6.389718, 62.357500, 61.388327)
+GRL	Greenland	ENVELOPE(-73.053609, -12.157637, 83.623600, 59.790273)
+XGK	Guernsey	ENVELOPE(-2.668609, -2.500973, 49.508191, 49.422491)
+ISL	Iceland	ENVELOPE(-24.538400, -13.499446, 66.536100, 63.390000)
+IRL	Ireland	ENVELOPE(-10.474727, -6.013055, 55.379991, 51.445545)
+XIM	Isle of Man	ENVELOPE(-4.787155, -4.308682, 54.416382, 54.055545)
+SJM	Jan Mayen	ENVELOPE(-9.119909, -7.928509, 71.180818, 70.803863)
+XJE	Jersey	ENVELOPE(-2.247364, -2.015000, 49.261109, 49.167773)
+GBR	United Kingdom	ENVELOPE(-8.171664, 1.749445, 60.843327, 49.955273)
+CPV	Cape Verde	ENVELOPE(-25.360555, -22.666109, 17.192364, 14.811109)
+CIV	Cote d'Ivoire	ENVELOPE(-8.606382, -2.487782, 10.735254, 4.344718)
+GHA	Ghana	ENVELOPE(-3.248891, 1.202782, 11.155691, 4.727082)
+GIB	Gibraltar	ENVELOPE(-5.356173, -5.334509, 36.163309, 36.112073)
+LBR	Liberia	ENVELOPE(-11.492327, -7.368400, 8.512782, 4.343609)
+MAR	Morocco	ENVELOPE(-13.174964, -1.011809, 35.919164, 27.664236)
+PRT	Portugal	ENVELOPE(-31.289027, -6.190455, 42.150673, 32.637500)
+ESP	Spain	ENVELOPE(-18.169864, 4.316945, 43.764300, 27.637500)
+ESH	Western Sahara	ENVELOPE(-17.101527, -8.666391, 27.666954, 20.764100)
+BFA	Burkina Faso	ENVELOPE(-5.520837, 2.397927, 15.082773, 9.395691)
+GIN	Guinea	ENVELOPE(-15.080837, -7.653373, 12.677500, 7.193927)
+GNB	Guinea-Bissau	ENVELOPE(-16.717773, -13.643891, 12.684718, 10.925100)
+MLI	Mali	ENVELOPE(-12.244837, 4.251391, 25.000273, 10.142154)
+MRT	Mauritania	ENVELOPE(-17.075555, -4.806109, 27.290454, 14.725636)
+SEN	Senegal	ENVELOPE(-17.532782, -11.369927, 16.690618, 12.301745)
+SLE	Sierra Leone	ENVELOPE(-13.295609, -10.264309, 9.997500, 6.923609)
+GMB	The Gambia	ENVELOPE(-16.821664, -13.798609, 13.826391, 13.059973)
+DJI	Djibouti	ENVELOPE(41.759854, 43.420409, 12.708327, 10.942218)
+ERI	Eritrea	ENVELOPE(36.443282, 43.121382, 17.994882, 12.363891)
+ETH	Ethiopia	ENVELOPE(32.991800, 47.988245, 14.883609, 3.406664)
+MNG	Mongolia	ENVELOPE(87.761100, 119.931509, 52.142773, 41.586654)
+SDN	Sudan	ENVELOPE(21.829100, 38.607500, 22.232218, 3.493391)
+UGA	Uganda	ENVELOPE(29.574300, 35.009718, 4.222782, -1.476109)
+ISR	Gaza Strip	ENVELOPE(34.216663, 34.558891, 31.596100, 31.216545)
+IRQ	Iraq	ENVELOPE(38.794700, 48.560691, 37.383673, 29.061664)
+ISR	Israel	ENVELOPE(34.267582, 35.681109, 33.270273, 29.486709)
+JOR	Jordan	ENVELOPE(34.960418, 39.301109, 33.377591, 29.188891)
+KAZ	Kazakhstan	ENVELOPE(46.499163, 87.348209, 55.442627, 40.594436)
+NOR	Norway	ENVELOPE(4.789582, 31.073536, 71.154709, 57.987918)
+RUS	Russia	ENVELOPE(-180.000000, 180.000000, 81.851927, 41.196582)
+SWE	Sweden	ENVELOPE(11.113336, 24.167009, 69.060300, 55.339164)
+ISR	West Bank	ENVELOPE(34.888191, 35.570609, 32.546391, 31.350691)
+DZA	Algeria	ENVELOPE(-8.667218, 11.986473, 37.089854, 18.976391)
+AND	Andorra	ENVELOPE(1.421391, 1.781718, 42.655964, 42.436382)
+CMR	Cameroon	ENVELOPE(8.502363, 16.207000, 13.085000, 1.654164)
+CAF	Central African Republic	ENVELOPE(14.418891, 27.459718, 11.000836, 2.221264)
+LBY	Libya	ENVELOPE(9.311391, 25.151663, 33.171136, 19.499064)
+MCO	Monaco	ENVELOPE(7.390900, 7.439291, 43.768300, 43.727545)
+TUN	Tunisia	ENVELOPE(7.492218, 11.581663, 37.340409, 30.234391)
+BEN	Benin	ENVELOPE(0.776663, 3.855000, 12.396654, 6.218718)
+TCD	Chad	ENVELOPE(13.461945, 24.002745, 23.450554, 7.458536)
+GNQ	Equatorial Guinea	ENVELOPE(8.424163, 11.353891, 3.763336, 0.930154)
+KIR	Kiribati	ENVELOPE(-157.581700, 172.947509, 2.033054, 1.335991)
+NER	Niger	ENVELOPE(0.166663, 15.996663, 23.522309, 11.693273)
+NGA	Nigeria	ENVELOPE(2.692500, 14.649654, 13.891500, 4.272845)
+STP	Sao Tome & Principe	ENVELOPE(6.465136, 7.463473, 1.701245, 0.018336)
+TGO	Togo	ENVELOPE(-0.149764, 1.797800, 11.138536, 6.100545)
+ALB	Albania	ENVELOPE(19.288536, 21.053327, 42.660345, 39.645000)
+BIH	Bosnia & Herzegovina	ENVELOPE(15.740591, 19.619782, 45.265945, 42.565827)
+HRV	Croatia	ENVELOPE(13.504791, 19.425000, 46.535827, 42.399991)
+ITA	Italy	ENVELOPE(6.623963, 18.514445, 47.094582, 36.649164)
+MKD	Macedonia	ENVELOPE(20.458818, 23.030973, 42.358954, 40.855891)
+MLT	Malta	ENVELOPE(14.329100, 14.570000, 35.991936, 35.800000)
+SMR	San Marino	ENVELOPE(12.406945, 12.511109, 43.986873, 43.898682)
+SMN	Serbia & Montenegro	ENVELOPE(18.453327, 23.005000, 46.181109, 41.849000)
+VTC	Vatican City	ENVELOPE(12.444473, 12.457718, 41.908391, 41.900891)
+BGR	Bulgaria	ENVELOPE(22.365273, 28.605136, 44.224718, 41.243045)
+CYP	Cyprus	ENVELOPE(32.269863, 34.586036, 35.688609, 34.640273)
+EGY	Egypt	ENVELOPE(24.706800, 36.895827, 31.646945, 21.994164)
+GEO	Georgia	ENVELOPE(40.002963, 46.710818, 43.584718, 41.048045)
+GRC	Greece	ENVELOPE(19.640000, 28.238045, 41.747773, 34.930545)
+LBN	Lebanon	ENVELOPE(35.100827, 36.623745, 34.647500, 33.062082)
+SYR	Syria	ENVELOPE(35.614463, 42.378327, 37.290545, 32.313609)
+TUR	Turkey	ENVELOPE(25.665827, 44.820545, 42.109991, 35.818445)
+AUT	Austria	ENVELOPE(9.533573, 17.166382, 49.018745, 46.407491)
+CZE	Czech Republic	ENVELOPE(12.093700, 18.852218, 51.052491, 48.581382)
+DNK	Denmark	ENVELOPE(8.092918, 15.149163, 57.745973, 54.561936)
+HUN	Hungary	ENVELOPE(16.111800, 22.894800, 48.576173, 45.748327)
+POL	Poland	ENVELOPE(14.147636, 24.143473, 54.836036, 49.002918)
+SVK	Slovakia	ENVELOPE(16.844718, 22.558054, 49.600827, 47.737500)
+SVN	Slovenia	ENVELOPE(13.383473, 16.607873, 46.876245, 45.425818)
+SJM	Svalbard	ENVELOPE(10.487918, 33.637500, 80.764163, 74.343045)
+BEL	Belgium	ENVELOPE(2.541663, 6.398200, 51.501245, 49.508882)
+FRA	France	ENVELOPE(-4.790282, 9.562218, 51.091109, 41.364927)
+DEU	Germany	ENVELOPE(5.865000, 15.033818, 55.056527, 47.274718)
+LIE	Liechtenstein	ENVELOPE(9.474636, 9.633891, 47.274545, 47.057454)
+LUX	Luxembourg	ENVELOPE(5.734445, 6.524027, 50.181809, 49.448464)
+NLD	Netherlands	ENVELOPE(3.370863, 7.210973, 53.465827, 50.753882)
+CHE	Switzerland	ENVELOPE(5.967009, 10.488209, 47.806664, 45.829436)
+USA	United States	ENVELOPE(-178.216555, 179.775936, 71.351436, 18.925482)
+BLR	Belarus	ENVELOPE(23.165400, 32.740054, 56.167491, 51.251845)
+EST	Estonia	ENVELOPE(21.837354, 28.194091, 59.664718, 57.522636)
+FIN	Finland	ENVELOPE(19.511391, 31.581963, 70.088609, 59.806800)
+LVA	Latvia	ENVELOPE(20.968609, 28.235963, 58.083254, 55.674836)
+LTU	Lithuania	ENVELOPE(20.942836, 26.813054, 56.449854, 53.890336)
+MDA	Moldova	ENVELOPE(26.634991, 30.128709, 48.468318, 45.448645)
+ROM	Romania	ENVELOPE(20.261027, 29.672218, 48.263882, 43.623309)
+UKR	Ukraine	ENVELOPE(22.151445, 40.178745, 52.378600, 44.379154)
+IND	India	ENVELOPE(68.144227, 97.380536, 35.505618, 6.745827)
+MDV	Maldives	ENVELOPE(72.863391, 73.637272, 7.027773, -0.641664)
+OMN	Oman	ENVELOPE(51.999291, 59.847082, 26.368709, 16.642782)
+SOM	Somalia	ENVELOPE(40.988609, 51.411318, 11.979164, -1.674873)
+LKA	Sri Lanka	ENVELOPE(79.696091, 81.891663, 9.828191, 5.918054)
+TKM	Turkmenistan	ENVELOPE(51.250182, 66.670882, 42.796173, 35.145991)
+UZB	Uzbekistan	ENVELOPE(55.997491, 73.167545, 45.570591, 37.184991)
+YEM	Yemen	ENVELOPE(42.555973, 54.473473, 18.999345, 12.144718)
+ARM	Armenia	ENVELOPE(43.454163, 46.620536, 41.297054, 38.841145)
+AZE	Azerbaijan	ENVELOPE(44.778863, 51.677009, 42.710754, 38.262809)
+BHR	Bahrain	ENVELOPE(50.453327, 50.796391, 26.288891, 25.571945)
+IRN	Iran	ENVELOPE(44.034954, 63.330273, 39.779154, 25.075973)
+KWT	Kuwait	ENVELOPE(46.546945, 48.416591, 30.084164, 28.538882)
+QAT	Qatar	ENVELOPE(50.751936, 51.615827, 26.152500, 24.556045)
+SAU	Saudi Arabia	ENVELOPE(34.572145, 55.666109, 32.154945, 16.377500)
+ARE	United Arab Emirates	ENVELOPE(51.583327, 56.381663, 26.083882, 22.633327)
+AFG	Afghanistan	ENVELOPE(60.504163, 74.915736, 38.471982, 29.406109)
+KGZ	Kyrgyzstan	ENVELOPE(69.249500, 80.281582, 43.216900, 39.195473)
+NPL	Nepal	ENVELOPE(80.052200, 88.194554, 30.424718, 26.368364)
+PAK	Pakistan	ENVELOPE(60.866300, 77.823927, 37.060791, 23.688045)
+TJK	Tajikistan	ENVELOPE(67.364700, 75.187482, 41.049254, 36.671845)
+BGD	Bangladesh	ENVELOPE(88.043872, 92.669345, 26.626136, 20.744818)
+BTN	Bhutan	ENVELOPE(88.751936, 92.114218, 28.325000, 26.703609)
+BRN	Brunei	ENVELOPE(114.095082, 115.360263, 5.053054, 4.018191)
+CHN	China	ENVELOPE(73.620045, 134.768463, 53.553745, 18.168882)
+JPN	Japan	ENVELOPE(123.678863, 145.812409, 45.486382, 24.251391)
+PRK	North Korea	ENVELOPE(124.323954, 130.697418, 43.006100, 37.671382)
+PLW	Palau	ENVELOPE(134.452482, 134.658872, 7.729445, 7.305254)
+PHL	Philippines	ENVELOPE(116.950000, 126.598036, 19.391109, 5.049164)
+KOR	South Korea	ENVELOPE(126.099018, 129.586872, 38.625245, 33.192209)
+KHM	Cambodia	ENVELOPE(102.346509, 107.636382, 14.708618, 10.422736)
+LAO	Laos	ENVELOPE(100.091372, 107.695254, 22.499927, 13.926664)
+MYS	Malaysia	ENVELOPE(99.641936, 119.275818, 7.352918, 0.852782)
+MMR	Myanmar	ENVELOPE(92.204991, 101.169427, 28.546527, 9.839582)
+SGP	Singapore	ENVELOPE(103.640945, 103.997945, 1.445282, 1.259027)
+THA	Thailand	ENVELOPE(97.347272, 105.639291, 20.454582, 5.633473)
+VNM	Vietnam	ENVELOPE(102.140745, 109.464845, 23.324164, 8.559236)
+GUM	Guam	ENVELOPE(144.634154, 144.953309, 13.652291, 13.235000)
+MHL	Marshall Is.	ENVELOPE(162.324963, 171.378063, 14.594027, 5.600273)
+FSM	Micronesia	ENVELOPE(158.120100, 163.042891, 6.977636, 5.261664)
+MNP	Northern Mariana Is.	ENVELOPE(145.572682, 145.818082, 15.268191, 14.908054)
+UMI	Wake I.	ENVELOPE(166.608981, 166.662200, 19.324582, 19.279445)
+BWA	Botswana	ENVELOPE(19.996109, 29.373618, -17.782082, -26.875555)
+BDI	Burundi	ENVELOPE(28.985000, 30.853191, -2.301564, -4.448055)
+ATF	French Southern & Antarctic Lands	ENVELOPE(51.650836, 70.567491, -46.327645, -49.725009)
+HMD	Heard I. & McDonald Is.	ENVELOPE(73.234709, 73.773882, -52.965145, -53.199445)
+KEN	Kenya	ENVELOPE(33.907218, 41.905163, 4.622500, -4.669618)
+RWA	Rwanda	ENVELOPE(28.854445, 30.893263, -1.054446, -2.825491)
+TZA	Tanzania	ENVELOPE(29.340827, 40.436809, -0.997218, -11.740418)
+ZMB	Zambia	ENVELOPE(21.996391, 33.702282, -8.191664, -18.074918)
+ZWE	Zimbabwe	ENVELOPE(25.237918, 33.071591, -15.616527, -22.414764)
+ATA	Antarctica	ENVELOPE(-180.000000, 180.000000, -60.503336, -90.000000)
+NOR	Bouvet I.	ENVELOPE(3.342363, 3.484163, -54.383609, -54.462782)
+COM	Comoros	ENVELOPE(43.214027, 44.530418, -11.366946, -12.383055)
+REU	Juan De Nova I.	ENVELOPE(42.723818, 42.760900, -17.052018, -17.076118)
+LSO	Lesotho	ENVELOPE(27.013973, 29.455554, -28.570691, -30.650527)
+MWI	Malawi	ENVELOPE(32.681873, 35.920963, -9.376673, -17.135282)
+MOZ	Mozambique	ENVELOPE(30.213018, 40.846109, -10.471109, -26.860282)
+ZAF	South Africa	ENVELOPE(16.483327, 37.892218, -22.136391, -46.969727)
+SWZ	Swaziland	ENVELOPE(30.798336, 32.133400, -25.728336, -27.316391)
+AGO	Angola	ENVELOPE(11.731245, 24.084445, -4.388991, -18.016391)
+COG	Congo	ENVELOPE(11.140663, 18.643609, 3.711109, -5.015000)
+ZAR	Congo, DRC	ENVELOPE(12.214554, 31.302773, 5.380691, -13.458055)
+FJI	Fiji	ENVELOPE(-180.000000, 180.000000, -16.153473, -19.162782)
+GAB	Gabon	ENVELOPE(8.700836, 14.519582, 2.317900, -3.925282)
+NAM	Namibia	ENVELOPE(11.716391, 25.264427, -16.954173, -28.961873)
+NZL	New Zealand	ENVELOPE(-176.848755, 178.841063, -34.414718, -52.578055)
+IOT	British Indian Ocean Territory	ENVELOPE(72.357900, 72.494282, -7.233473, -7.436246)
+REU	Glorioso Is.	ENVELOPE(47.279091, 47.303054, -11.554100, -11.577782)
+MDG	Madagascar	ENVELOPE(43.236827, 50.501391, -11.945555, -25.588336)
+MUS	Mauritius	ENVELOPE(57.306309, 63.495754, -19.673336, -20.520555)
+MYT	Mayotte	ENVELOPE(45.039163, 45.293345, -12.662500, -12.992500)
+REU	Reunion	ENVELOPE(55.220554, 55.853054, -20.856527, -21.373891)
+SYC	Seychelles	ENVELOPE(46.205691, 55.540554, -4.551664, -9.463055)
+CXR	Christmas I.	ENVELOPE(105.629000, 105.751900, -10.384082, -10.510973)
+CCK	Cocos Is.	ENVELOPE(96.817491, 96.864845, -12.130418, -12.199446)
+IDN	Indonesia	ENVELOPE(95.210945, 141.007018, 5.913473, -10.929655)
+TLS	Timor Leste	ENVELOPE(124.046100, 127.308591, -8.140000, -9.463627)
+AUS	Australia	ENVELOPE(112.907209, 158.960372, -10.135691, -54.753891)
+NRU	Nauru	ENVELOPE(166.904418, 166.957045, -0.493336, -0.552218)
+NCL	New Caledonia	ENVELOPE(163.982745, 168.130509, -20.087918, -22.673891)
+NFK	Norfolk I.	ENVELOPE(167.910945, 167.998872, -29.000555, -29.081109)
+PNG	Papua New Guinea	ENVELOPE(140.858854, 155.966845, -1.355282, -11.642500)
+SLB	Solomon Is.	ENVELOPE(155.671300, 166.931836, -6.605518, -11.845836)
+TUV	Tuvalu	ENVELOPE(176.295254, 179.232281, -6.089446, -8.561291)
+VUT	Vanuatu	ENVELOPE(166.521636, 169.893863, -13.707218, -20.254173)


[17/26] lucenenet git commit: Repaired several issues that were causing tests to fail. Refactored some of the API, added some missing documentation, and set member accessibility on some members.

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialOperation.cs b/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
index 8d33ecf..5f5aa35 100644
--- a/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
+++ b/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
@@ -13,6 +13,7 @@
  * limitations under the License.
  */
 
+using Lucene.Net.Support;
 using Spatial4n.Core.Shapes;
 using System;
 using System.Collections.Generic;
@@ -21,11 +22,21 @@ using System.Linq;
 
 namespace Lucene.Net.Spatial.Queries
 {
+    /// <summary>
+    /// A clause that compares a stored geometry to a supplied geometry. For more
+    /// explanation of each operation, consider looking at the source implementation
+    /// of <see cref="Evaluate(IShape, IShape)"/>.
+    /// <para>
+    /// See <a href="http://edndoc.esri.com/arcsde/9.1/general_topics/understand_spatial_relations.htm">
+    /// ESRIs docs on spatial relations</a>
+    /// </para>
+    /// @lucene.experimental
+    /// </summary>
     [Serializable]
     public abstract class SpatialOperation
     {
         // Private registry
-        private static readonly Dictionary<string, SpatialOperation> registry = new Dictionary<string, SpatialOperation>();
+        private static readonly IDictionary<string, SpatialOperation> registry = new Dictionary<string, SpatialOperation>();
         private static readonly IList<SpatialOperation> list = new List<SpatialOperation>();
 
         // Geometry Operations
@@ -163,7 +174,7 @@ namespace Lucene.Net.Spatial.Queries
             this.sourceNeedsArea = sourceNeedsArea;
             this.targetNeedsArea = targetNeedsArea;
             registry[name] = this;
-            registry[name.ToUpper(CultureInfo.CreateSpecificCulture("en-US"))] = this;
+            registry[name.ToUpper(CultureInfo.InvariantCulture)] = this;
             list.Add(this);
         }
 
@@ -172,15 +183,15 @@ namespace Lucene.Net.Spatial.Queries
             SpatialOperation op;
             if (!registry.TryGetValue(v, out op) || op == null)
             {
-                if (!registry.TryGetValue(v.ToUpper(CultureInfo.CreateSpecificCulture("en-US")), out op) || op == null)
+                if (!registry.TryGetValue(v.ToUpper(CultureInfo.InvariantCulture), out op) || op == null)
                     throw new ArgumentException("Unknown Operation: " + v, "v");
             }
             return op;
         }
 
-        public static IList<SpatialOperation> Values()
+        public static IList<SpatialOperation> Values
         {
-            return list;
+            get { return list; }
         }
 
         public static bool Is(SpatialOperation op, params SpatialOperation[] tst)
@@ -188,26 +199,30 @@ namespace Lucene.Net.Spatial.Queries
             return tst.Any(t => op == t);
         }
 
+        /// <summary>
+        /// Returns whether the relationship between indexedShape and queryShape is
+        /// satisfied by this operation.
+        /// </summary>
         public abstract bool Evaluate(IShape indexedShape, IShape queryShape);
 
         // ================================================= Getters / Setters =============================================
 
-        public bool IsScoreIsMeaningful()
+        public virtual bool IsScoreIsMeaningful
         {
-            return scoreIsMeaningful;
+            get { return scoreIsMeaningful; }
         }
 
-        public bool IsSourceNeedsArea()
+        public virtual bool IsSourceNeedsArea
         {
-            return sourceNeedsArea;
+            get { return sourceNeedsArea; }
         }
 
-        public bool IsTargetNeedsArea()
+        public virtual bool IsTargetNeedsArea
         {
-            return targetNeedsArea;
+            get { return targetNeedsArea; }
         }
 
-        public string Name
+        public virtual string Name
         {
             get { return name; }
         }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs b/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs
index 35949bc..e79027f 100644
--- a/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs
+++ b/src/Lucene.Net.Spatial/Query/UnsupportedSpatialOperation.cs
@@ -20,7 +20,7 @@ using System;
 namespace Lucene.Net.Spatial.Queries
 {
     [Serializable]
-    public class UnsupportedSpatialOperation : InvalidOperationException
+    public class UnsupportedSpatialOperation : NotSupportedException
     {
         public UnsupportedSpatialOperation(SpatialOperation op)
             : base(op.Name)

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
index f3413b2..2358f19 100644
--- a/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
+++ b/src/Lucene.Net.Spatial/Serialized/SerializedDVStrategy.cs
@@ -136,11 +136,11 @@ namespace Lucene.Net.Spatial.Serialized
             return new PredicateValueSourceFilter(predicateValueSource);
         }
 
-        /**
-        * Provides access to each shape per document as a ValueSource in which
-        * {@link org.apache.lucene.queries.function.FunctionValues#objectVal(int)} returns a {@link
-        * Shape}.
-        */ //TODO raise to SpatialStrategy
+        /// <summary>
+        /// Provides access to each shape per document as a ValueSource in which
+        /// <see cref="FunctionValues.ObjectVal(int)"/> returns a <see cref="IShape"/>.
+        /// </summary>
+        //TODO raise to SpatialStrategy
         public virtual ValueSource MakeShapeValueSource()
         {
             return new ShapeDocValueSource(this, FieldName, ctx.BinaryCodec);
@@ -148,7 +148,7 @@ namespace Lucene.Net.Spatial.Serialized
 
         /// <summary>
         /// This filter only supports returning a DocSet with a GetBits(). If you try to grab the
-        /// iterator then you'll get an UnsupportedOperationException.
+        /// iterator then you'll get a <see cref="NotSupportedException"/>.
         /// </summary>
         internal class PredicateValueSourceFilter : Filter
         {
@@ -276,10 +276,10 @@ namespace Lucene.Net.Spatial.Serialized
             }
         }//PredicateValueSourceFilter
 
-        /**
-         * Implements a ValueSource by deserializing a Shape in from BinaryDocValues using BinaryCodec.
-         * @see #makeShapeValueSource()
-         */
+        /// <summary>
+        /// Implements a <see cref="ValueSource"/> by deserializing a <see cref="IShape"/> in from <see cref="BinaryDocValues"/> using <see cref="BinaryCodec"/>.
+        /// </summary>
+        /// <seealso cref="MakeShapeValueSource()"/>
         internal class ShapeDocValueSource : ValueSource
         {
             private readonly SerializedDVStrategy outerInstance;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/SpatialStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/SpatialStrategy.cs b/src/Lucene.Net.Spatial/SpatialStrategy.cs
index 5f16c71..3c80f68 100644
--- a/src/Lucene.Net.Spatial/SpatialStrategy.cs
+++ b/src/Lucene.Net.Spatial/SpatialStrategy.cs
@@ -28,11 +28,26 @@ namespace Lucene.Net.Spatial
 {
     /// <summary>
     /// The SpatialStrategy encapsulates an approach to indexing and searching based on shapes.
-    /// <p/>
+    /// <para/>
+    /// Different implementations will support different features. A strategy should
+    /// document these common elements:
+    /// <list type="bullet">
+    ///     <item>Can it index more than one shape per field?</item>
+    ///     <item>What types of shapes can be indexed?</item>
+    ///     <item>What types of query shapes can be used?</item>
+    ///     <item>What types of query operations are supported? This might vary per shape.</item>
+    ///     <item>Does it use the <see cref="FieldCache"/>, or some other type of cache?  When?</item>
+    /// </list>
+    /// If a strategy only supports certain shapes at index or query time, then in
+    /// general it will throw an exception if given an incompatible one.  It will not
+    /// be coerced into compatibility.
+    /// <para/>
     /// Note that a SpatialStrategy is not involved with the Lucene stored field values of shapes, which is
     /// immaterial to indexing and search.
-    /// <p/>
+    /// <para/>
     /// Thread-safe.
+    /// 
+    /// @lucene.experimental
     /// </summary>
     public abstract class SpatialStrategy
     {
@@ -42,8 +57,6 @@ namespace Lucene.Net.Spatial
         /// <summary>
         /// Constructs the spatial strategy with its mandatory arguments.
         /// </summary>
-        /// <param name="ctx"></param>
-        /// <param name="fieldName"> </param>
         protected SpatialStrategy(SpatialContext ctx, string fieldName)
         {
             if (ctx == null)
@@ -54,7 +67,7 @@ namespace Lucene.Net.Spatial
             this.fieldName = fieldName;
         }
 
-        public SpatialContext SpatialContext
+        public virtual SpatialContext SpatialContext
         {
             get { return ctx; }
         }
@@ -64,7 +77,7 @@ namespace Lucene.Net.Spatial
         /// fields needed internally.
         /// </summary>
         /// <returns></returns>
-        public String FieldName
+        public virtual string FieldName
         {
             get { return fieldName; }
         }
@@ -90,7 +103,7 @@ namespace Lucene.Net.Spatial
         /// </summary>
         /// <param name="queryPoint"></param>
         /// <returns></returns>
-        public ValueSource MakeDistanceValueSource(IPoint queryPoint)
+        public virtual ValueSource MakeDistanceValueSource(IPoint queryPoint)
         {
             return MakeDistanceValueSource(queryPoint, 1.0);
         }
@@ -136,8 +149,6 @@ namespace Lucene.Net.Spatial
         /// 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(IShape queryShape)
         {
             IRectangle bbox = queryShape.BoundingBox;
@@ -145,7 +156,7 @@ namespace Lucene.Net.Spatial
                 ctx.MakePoint(bbox.MinX, bbox.MinY), bbox.MaxX, bbox.MaxY);
             double distToEdge = diagonalDist * 0.5;
             float c = (float)distToEdge * 0.1f; //one tenth
-            return new ReciprocalFloatFunction(MakeDistanceValueSource(queryShape.Center), 1f, c, c);
+            return new ReciprocalFloatFunction(MakeDistanceValueSource(queryShape.Center, 1.0), 1f, c, c);
         }
 
         public override string ToString()

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Util/CachingDoubleValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/CachingDoubleValueSource.cs b/src/Lucene.Net.Spatial/Util/CachingDoubleValueSource.cs
index 793e10f..a68ed73 100644
--- a/src/Lucene.Net.Spatial/Util/CachingDoubleValueSource.cs
+++ b/src/Lucene.Net.Spatial/Util/CachingDoubleValueSource.cs
@@ -30,9 +30,9 @@ namespace Lucene.Net.Spatial.Util
     /// </summary>
     public class CachingDoubleValueSource : ValueSource
     {
-        protected readonly IDictionary<int, double> cache;
         protected readonly ValueSource source;
-
+        protected readonly IDictionary<int, double> cache;
+        
         public CachingDoubleValueSource(ValueSource source)
         {
             this.source = source;
@@ -51,26 +51,9 @@ namespace Lucene.Net.Spatial.Util
             return new CachingDoubleFunctionValue(@base, vals, cache);
         }
 
-        public override bool Equals(object o)
-        {
-            if (this == o) return true;
-
-            var that = o as CachingDoubleValueSource;
-
-            if (that == null) return false;
-            if (source != null ? !source.Equals(that.source) : that.source != null) return false;
-
-            return true;
-        }
-
-        public override int GetHashCode()
-        {
-            return source != null ? source.GetHashCode() : 0;
-        }
-
         #region Nested type: CachingDoubleFunctionValue
 
-        public class CachingDoubleFunctionValue : FunctionValues
+        internal class CachingDoubleFunctionValue : FunctionValues
         {
             private readonly IDictionary<int, double> cache;
             private readonly int docBase;
@@ -107,5 +90,22 @@ namespace Lucene.Net.Spatial.Util
         }
 
         #endregion
+
+        public override bool Equals(object o)
+        {
+            if (this == o) return true;
+
+            var that = o as CachingDoubleValueSource;
+
+            if (that == null) return false;
+            if (source != null ? !source.Equals(that.source) : that.source != null) return false;
+
+            return true;
+        }
+
+        public override int GetHashCode()
+        {
+            return source != null ? source.GetHashCode() : 0;
+        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Util/DistanceToShapeValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/DistanceToShapeValueSource.cs b/src/Lucene.Net.Spatial/Util/DistanceToShapeValueSource.cs
index 721677c..76fc7bd 100644
--- a/src/Lucene.Net.Spatial/Util/DistanceToShapeValueSource.cs
+++ b/src/Lucene.Net.Spatial/Util/DistanceToShapeValueSource.cs
@@ -65,6 +65,34 @@ namespace Lucene.Net.Spatial.Util
             shapeValueSource.CreateWeight(context, searcher);
         }
 
+        public override FunctionValues GetValues(IDictionary context, AtomicReaderContext readerContext)
+        {
+            FunctionValues shapeValues = shapeValueSource.GetValues(context, readerContext);
+
+            return new DoubleDocValuesAnonymousHelper(this, shapeValues);
+
+            //return new DoubleDocValues(this)
+            //    {
+            //        @Override
+            //  public double doubleVal(int doc)
+            //    {
+            //        Shape shape = (Shape)shapeValues.objectVal(doc);
+            //        if (shape == null || shape.isEmpty())
+            //            return nullValue;
+            //        Point pt = shape.getCenter();
+            //        return distCalc.distance(queryPoint, pt) * multiplier;
+            //    }
+
+            //    @Override
+            //  public Explanation explain(int doc)
+            //    {
+            //        Explanation exp = super.explain(doc);
+            //        exp.addDetail(shapeValues.explain(doc));
+            //        return exp;
+            //    }
+            //};
+        }
+
         internal class DoubleDocValuesAnonymousHelper : DoubleDocValues
         {
             private readonly DistanceToShapeValueSource outerInstance;
@@ -94,34 +122,6 @@ namespace Lucene.Net.Spatial.Util
             }
         }
 
-        public override FunctionValues GetValues(IDictionary context, AtomicReaderContext readerContext)
-        {
-            FunctionValues shapeValues = shapeValueSource.GetValues(context, readerContext);
-
-            return new DoubleDocValuesAnonymousHelper(this, shapeValues);
-
-            //return new DoubleDocValues(this)
-            //    {
-            //        @Override
-            //  public double doubleVal(int doc)
-            //    {
-            //        Shape shape = (Shape)shapeValues.objectVal(doc);
-            //        if (shape == null || shape.isEmpty())
-            //            return nullValue;
-            //        Point pt = shape.getCenter();
-            //        return distCalc.distance(queryPoint, pt) * multiplier;
-            //    }
-
-            //    @Override
-            //  public Explanation explain(int doc)
-            //    {
-            //        Explanation exp = super.explain(doc);
-            //        exp.addDetail(shapeValues.explain(doc));
-            //        return exp;
-            //    }
-            //};
-        }
-
         public override bool Equals(object o)
         {
             if (this == o) return true;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs b/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs
index 23d04c4..96cfb3e 100644
--- a/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs
+++ b/src/Lucene.Net.Spatial/Util/ShapeFieldCache.cs
@@ -23,6 +23,11 @@ namespace Lucene.Net.Spatial.Util
     /// <summary>
     /// Bounded Cache of Shapes associated with docIds.  Note, multiple Shapes can be
     /// associated with a given docId
+    /// <para/>
+    /// WARNING: This class holds the data in an extremely inefficient manner as all Points are in memory as objects and they
+    /// are stored in many ArrayLists (one per document).  So it works but doesn't scale.  It will be replaced in the future.
+    /// <para/>
+    /// @lucene.internal
     /// </summary>
     /// <typeparam name="T"></typeparam>
     public class ShapeFieldCache<T> where T : IShape
@@ -36,7 +41,7 @@ namespace Lucene.Net.Spatial.Util
             this.defaultLength = defaultLength;
         }
 
-        public void Add(int docid, T s)
+        public virtual void Add(int docid, T s)
         {
             IList<T> list = cache[docid];
             if (list == null)
@@ -46,10 +51,9 @@ namespace Lucene.Net.Spatial.Util
             list.Add(s);
         }
 
-        public IList<T> GetShapes(int docid)
+        public virtual IList<T> GetShapes(int docid)
         {
             return cache[docid];
         }
-
     }
 }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs b/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
index c5f4af7..144424e 100644
--- a/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
+++ b/src/Lucene.Net.Spatial/Util/ShapeFieldCacheDistanceValueSource.cs
@@ -27,13 +27,18 @@ using Spatial4n.Core.Shapes;
 namespace Lucene.Net.Spatial.Util
 {
     /// <summary>
-    /// An implementation of the Lucene ValueSource model to support spatial relevance ranking.
+    /// An implementation of the Lucene ValueSource that returns the spatial distance
+    /// between an input point and a document's points in
+    /// <see cref="ShapeFieldCacheProvider{T}"/>. The shortest distance is returned if a
+    /// document has more than one point.
+    /// 
+    /// @lucene.internal
     /// </summary>
     public class ShapeFieldCacheDistanceValueSource : ValueSource
     {
-        private readonly ShapeFieldCacheProvider<IPoint> provider;
         private readonly SpatialContext ctx;
         private readonly IPoint from;
+        private readonly ShapeFieldCacheProvider<IPoint> provider;
         private readonly double multiplier;
 
         public ShapeFieldCacheDistanceValueSource(SpatialContext ctx, 
@@ -45,7 +50,17 @@ namespace Lucene.Net.Spatial.Util
             this.multiplier = multiplier;
         }
 
-        public class CachedDistanceFunctionValue : FunctionValues
+        public override string Description
+        {
+            get { return GetType().Name + "(" + provider + ", " + from + ")"; }
+        }
+
+        public override FunctionValues GetValues(IDictionary context, AtomicReaderContext readerContext)
+        {
+            return new CachedDistanceFunctionValue(readerContext.AtomicReader, this);
+        }
+
+        internal class CachedDistanceFunctionValue : FunctionValues
         {
             private readonly ShapeFieldCacheDistanceValueSource enclosingInstance;
             private readonly ShapeFieldCache<IPoint> cache;
@@ -89,22 +104,10 @@ namespace Lucene.Net.Spatial.Util
             }
         }
 
-        public override string Description
-        {
-            get
-            {
-                return GetType().Name + "(" + provider + ", " + from + ")";
-            }
-        }
-
-        public override FunctionValues GetValues(IDictionary context, AtomicReaderContext readerContext)
-        {
-            return new CachedDistanceFunctionValue(readerContext.AtomicReader, this);
-        }
-
         public override bool Equals(object o)
         {
             if (this == o) return true;
+            if (o == null || GetType() != o.GetType()) return false;
 
             var that = o as ShapeFieldCacheDistanceValueSource;
 
@@ -112,6 +115,7 @@ namespace Lucene.Net.Spatial.Util
             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;
         }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Util/ShapeFieldCacheProvider.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ShapeFieldCacheProvider.cs b/src/Lucene.Net.Spatial/Util/ShapeFieldCacheProvider.cs
index 1a78210..45cb714 100644
--- a/src/Lucene.Net.Spatial/Util/ShapeFieldCacheProvider.cs
+++ b/src/Lucene.Net.Spatial/Util/ShapeFieldCacheProvider.cs
@@ -14,15 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-using System;
-using System.Runtime.CompilerServices;
 using Lucene.Net.Index;
 using Lucene.Net.Search;
+using Lucene.Net.Support;
 using Lucene.Net.Util;
 using Spatial4n.Core.Shapes;
-#if NET35
-using Lucene.Net.Support;
-#endif
 
 namespace Lucene.Net.Spatial.Util
 {
@@ -30,29 +26,27 @@ namespace Lucene.Net.Spatial.Util
     /// Provides access to a
     /// <see cref="ShapeFieldCache{T}">ShapeFieldCache&lt;T&gt;</see>
     /// for a given
-    /// <see cref="Lucene.Net.Index.AtomicReader">Lucene.Net.Index.AtomicReader
-    /// 	</see>
-    /// .
+    /// <see cref="Lucene.Net.Index.AtomicReader">Lucene.Net.Index.AtomicReader</see>.
+    /// 
     /// If a Cache does not exist for the Reader, then it is built by iterating over
     /// the all terms for a given field, reconstructing the Shape from them, and adding
     /// them to the Cache.
     /// </summary>
-    /// <lucene.internal></lucene.internal>
+    /// @lucene.internal
     public abstract class ShapeFieldCacheProvider<T>
         where T : IShape
     {
         //private Logger log = Logger.GetLogger(GetType().FullName);
 
 #if !NET35
-        private readonly ConditionalWeakTable<IndexReader, ShapeFieldCache<T>> sidx =
-            new ConditionalWeakTable<IndexReader, ShapeFieldCache<T>>(); // WeakHashMap
+        private readonly WeakDictionary<IndexReader, ShapeFieldCache<T>> sidx =
+            new WeakDictionary<IndexReader, ShapeFieldCache<T>>();
 #else
 	    private readonly WeakDictionary<IndexReader, ShapeFieldCache<T>> sidx =
 	        new WeakDictionary<IndexReader, ShapeFieldCache<T>>();
 #endif
 
         protected internal readonly int defaultSize;
-
         protected internal readonly string shapeField;
 
         public ShapeFieldCacheProvider(string shapeField, int defaultSize)
@@ -66,7 +60,6 @@ namespace Lucene.Net.Spatial.Util
 
         private readonly object locker = new object();
 
-        /// <exception cref="System.IO.IOException"></exception>
         public virtual ShapeFieldCache<T> GetCache(AtomicReader reader)
         {
             lock (locker)
@@ -104,7 +97,7 @@ namespace Lucene.Net.Spatial.Util
                         term = te.Next();
                     }
                 }
-                sidx.Add(reader, idx);
+                sidx[reader] = idx;
                 /*long elapsed = Runtime.CurrentTimeMillis() - startTime;
                 log.Fine("Cached: [" + count + " in " + elapsed + "ms] " + idx);*/
                 return idx;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Util/ShapePredicateValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ShapePredicateValueSource.cs b/src/Lucene.Net.Spatial/Util/ShapePredicateValueSource.cs
index fd88f40..7f65893 100644
--- a/src/Lucene.Net.Spatial/Util/ShapePredicateValueSource.cs
+++ b/src/Lucene.Net.Spatial/Util/ShapePredicateValueSource.cs
@@ -65,34 +65,6 @@ namespace Lucene.Net.Spatial.Util
             shapeValuesource.CreateWeight(context, searcher);
         }
 
-        internal class BoolDocValuesAnonymousHelper : BoolDocValues
-        {
-            private readonly ShapePredicateValueSource outerInstance;
-            private readonly FunctionValues shapeValues;
-
-            public BoolDocValuesAnonymousHelper(ShapePredicateValueSource outerInstance, FunctionValues shapeValues)
-                : base(outerInstance)
-            {
-                this.outerInstance = outerInstance;
-                this.shapeValues = shapeValues;
-            }
-
-            public override bool BoolVal(int doc)
-            {
-                IShape indexedShape = (IShape)shapeValues.ObjectVal(doc);
-                if (indexedShape == null)
-                    return false;
-                return outerInstance.op.Evaluate(indexedShape, outerInstance.queryShape);
-            }
-
-            public override Explanation Explain(int doc)
-            {
-                Explanation exp = base.Explain(doc);
-                exp.AddDetail(shapeValues.Explain(doc));
-                return exp;
-            }
-        }
-
         public override FunctionValues GetValues(IDictionary context, AtomicReaderContext readerContext)
         {
             FunctionValues shapeValues = shapeValuesource.GetValues(context, readerContext);
@@ -120,6 +92,33 @@ namespace Lucene.Net.Spatial.Util
             //};
         }
 
+        internal class BoolDocValuesAnonymousHelper : BoolDocValues
+        {
+            private readonly ShapePredicateValueSource outerInstance;
+            private readonly FunctionValues shapeValues;
+
+            public BoolDocValuesAnonymousHelper(ShapePredicateValueSource outerInstance, FunctionValues shapeValues)
+                : base(outerInstance)
+            {
+                this.outerInstance = outerInstance;
+                this.shapeValues = shapeValues;
+            }
+
+            public override bool BoolVal(int doc)
+            {
+                IShape indexedShape = (IShape)shapeValues.ObjectVal(doc);
+                if (indexedShape == null)
+                    return false;
+                return outerInstance.op.Evaluate(indexedShape, outerInstance.queryShape);
+            }
+
+            public override Explanation Explain(int doc)
+            {
+                Explanation exp = base.Explain(doc);
+                exp.AddDetail(shapeValues.Explain(doc));
+                return exp;
+            }
+        }
 
         public override bool Equals(object o)
         {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs b/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
index bcd73a6..45ffdd6 100644
--- a/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
+++ b/src/Lucene.Net.Spatial/Util/ValueSourceFilter.cs
@@ -26,9 +26,12 @@ namespace Lucene.Net.Spatial.Util
     /// <summary>
     /// Filter that matches all documents where a valuesource is
     /// in between a range of <c>min</c> and <c>max</c> inclusive.
+    /// @lucene.internal
     /// </summary>
     public class ValueSourceFilter : Filter
     {
+        //TODO see https://issues.apache.org/jira/browse/LUCENE-4251  (move out of spatial & improve)
+
         readonly Filter startingFilter;
         readonly ValueSource source;
         public readonly double min;
@@ -49,27 +52,26 @@ namespace Lucene.Net.Spatial.Util
         public override DocIdSet GetDocIdSet(AtomicReaderContext context, Bits acceptDocs)
         {
             var values = source.GetValues(null, context);
-            return new ValueSourceFilteredDocIdSet(startingFilter.GetDocIdSet(context, acceptDocs), values, this);
+            return new ValueSourceFilteredDocIdSet(this, startingFilter.GetDocIdSet(context, acceptDocs), values);
         }
 
-        public class ValueSourceFilteredDocIdSet : FilteredDocIdSet
+        internal class ValueSourceFilteredDocIdSet : FilteredDocIdSet
         {
-            private readonly ValueSourceFilter enclosingFilter;
+            private readonly ValueSourceFilter outerInstance;
             private readonly FunctionValues values;
 
-            public ValueSourceFilteredDocIdSet(DocIdSet innerSet, FunctionValues values, ValueSourceFilter caller)
+            public ValueSourceFilteredDocIdSet(ValueSourceFilter outerInstance, DocIdSet innerSet, FunctionValues values)
                 : base(innerSet)
             {
-                this.enclosingFilter = caller;
+                this.outerInstance = outerInstance;
                 this.values = values;
             }
 
             protected override bool Match(int doc)
             {
                 double val = values.DoubleVal(doc);
-                return val >= enclosingFilter.min && val <= enclosingFilter.max;
+                return val >= outerInstance.min && val <= outerInstance.max;
             }
         }
-
     }
 }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs b/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
index 75785f5..a186558 100644
--- a/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
+++ b/src/Lucene.Net.Spatial/Vector/DistanceValueSource.cs
@@ -28,7 +28,10 @@ using Spatial4n.Core.Shapes;
 namespace Lucene.Net.Spatial.Vector
 {
     /// <summary>
-    /// An implementation of the Lucene ValueSource model that returns the distance.
+    /// An implementation of the Lucene ValueSource model that returns the distance
+    /// for a <see cref="PointVectorStrategy"/>.
+    /// 
+    /// @lucene.internal
     /// </summary>
     public class DistanceValueSource : ValueSource
     {
@@ -36,6 +39,9 @@ namespace Lucene.Net.Spatial.Vector
         private readonly PointVectorStrategy strategy;
         private readonly double multiplier;
 
+        /// <summary>
+        /// Constructor.
+        /// </summary>
         public DistanceValueSource(PointVectorStrategy strategy, IPoint from, double multiplier)
         {
             this.strategy = strategy;
@@ -43,60 +49,46 @@ namespace Lucene.Net.Spatial.Vector
             this.multiplier = multiplier;
         }
 
+        /// <summary>
+        /// Returns the ValueSource description.
+        /// </summary>
         public override string Description
         {
             get { return "DistanceValueSource(" + strategy + ", " + from + ")"; }
         }
 
+        /// <summary>
+        /// Returns the FunctionValues used by the function query.
+        /// </summary>
         public override FunctionValues GetValues(IDictionary context, AtomicReaderContext readerContext)
         {
             return new DistanceFunctionValue(this, readerContext.AtomicReader);
         }
 
-        public override bool Equals(object o)
-        {
-            if (this == o) return true;
-
-            var that = o as DistanceValueSource;
-            if (that == null) return false;
-
-            if (!from.Equals(that.from)) return false;
-            if (!strategy.Equals(that.strategy)) return false;
-
-            return true;
-        }
-
-        public override int GetHashCode()
-        {
-            return from.GetHashCode();
-        }
-
         #region Nested type: DistanceFunctionValues
 
-        public class DistanceFunctionValue : FunctionValues
+        internal class DistanceFunctionValue : FunctionValues
         {
+            private readonly DistanceValueSource outerInstance;
             private readonly IDistanceCalculator calculator;
-            private readonly DistanceValueSource enclosingInstance;
             private readonly IPoint from;
             private readonly double nullValue;
-            private readonly double multiplier;
 
             private readonly FieldCache.Doubles ptX, ptY;
             private readonly Bits validX, validY;
 
-            public DistanceFunctionValue(DistanceValueSource enclosingInstance, AtomicReader reader)
+            public DistanceFunctionValue(DistanceValueSource outerInstance, AtomicReader reader)
             {
-                this.enclosingInstance = enclosingInstance;
+                this.outerInstance = outerInstance;
 
-                ptX = FieldCache.DEFAULT.GetDoubles(reader, enclosingInstance.strategy.FieldNameX, true);
-                ptY = FieldCache.DEFAULT.GetDoubles(reader, enclosingInstance.strategy.FieldNameY, true);
-                validX = FieldCache.DEFAULT.GetDocsWithField(reader, enclosingInstance.strategy.FieldNameX);
-                validY = FieldCache.DEFAULT.GetDocsWithField(reader, enclosingInstance.strategy.FieldNameY);
+                ptX = FieldCache.DEFAULT.GetDoubles(reader, outerInstance.strategy.FieldNameX, true);
+                ptY = FieldCache.DEFAULT.GetDoubles(reader, outerInstance.strategy.FieldNameY, true);
+                validX = FieldCache.DEFAULT.GetDocsWithField(reader, outerInstance.strategy.FieldNameX);
+                validY = FieldCache.DEFAULT.GetDocsWithField(reader, outerInstance.strategy.FieldNameY);
 
-                from = enclosingInstance.from;
-                multiplier = enclosingInstance.multiplier;
-                calculator = enclosingInstance.strategy.SpatialContext.DistCalc;
-                nullValue = (enclosingInstance.strategy.SpatialContext.IsGeo ? 180 * multiplier : double.MaxValue);
+                from = outerInstance.from;
+                calculator = outerInstance.strategy.SpatialContext.DistCalc;
+                nullValue = (outerInstance.strategy.SpatialContext.IsGeo ? 180 * outerInstance.multiplier : double.MaxValue);
             }
 
             public override float FloatVal(int doc)
@@ -110,17 +102,37 @@ namespace Lucene.Net.Spatial.Vector
                 if (validX.Get(doc))
                 {
                     Debug.Assert(validY.Get(doc));
-                    return calculator.Distance(from, ptX.Get(doc), ptY.Get(doc)) * multiplier;
+                    return calculator.Distance(outerInstance.from, ptX.Get(doc), ptY.Get(doc)) * outerInstance.multiplier;
                 }
                 return nullValue;
             }
 
             public override string ToString(int doc)
             {
-                return enclosingInstance.Description + "=" + FloatVal(doc);
+                return outerInstance.Description + "=" + FloatVal(doc);
             }
         }
 
         #endregion
+
+        public override bool Equals(object o)
+        {
+            if (this == o) return true;
+            if (o == null || GetType() != o.GetType()) return false;
+
+            var that = o as DistanceValueSource;
+            if (that == null) return false;
+
+            if (!from.Equals(that.from)) return false;
+            if (!strategy.Equals(that.strategy)) return false;
+            if (multiplier != that.multiplier) return false;
+
+            return true;
+        }
+
+        public override int GetHashCode()
+        {
+            return from.GetHashCode();
+        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
index dcbe259..d46bff0 100644
--- a/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
+++ b/src/Lucene.Net.Spatial/Vector/PointVectorStrategy.cs
@@ -27,26 +27,26 @@ using Spatial4n.Core.Shapes;
 namespace Lucene.Net.Spatial.Vector
 {
     /// <summary>
-    /// Simple {@link SpatialStrategy} which represents Points in two numeric {@link DoubleField}s.
+    /// Simple <see cref="SpatialStrategy"/> which represents Points in two numeric <see cref="DoubleField"/>s.
     /// 
     /// Note, currently only Points can be indexed by this Strategy.  At query time, the bounding
-    /// box of the given Shape is used to create {@link NumericRangeQuery}s to efficiently
+    /// box of the given Shape is used to create <see cref="NumericRangeQuery"/>s to efficiently
     /// find Points within the Shape.
     /// 
     /// Due to the simple use of numeric fields, this Strategy provides support for sorting by
-    /// distance through {@link DistanceValueSource}
+    /// distance through <see cref="DistanceValueSource"/>
     /// </summary>
     public class PointVectorStrategy : SpatialStrategy
     {
-        public static String SUFFIX_X = "__x";
-        public static String SUFFIX_Y = "__y";
+        public static string SUFFIX_X = "__x";
+        public static string SUFFIX_Y = "__y";
 
-        private readonly String fieldNameX;
-        private readonly String fieldNameY;
+        private readonly string fieldNameX;
+        private readonly string fieldNameY;
 
         public int precisionStep = 8; // same as solr default
 
-        public PointVectorStrategy(SpatialContext ctx, String fieldNamePrefix)
+        public PointVectorStrategy(SpatialContext ctx, string fieldNamePrefix)
             : base(ctx, fieldNamePrefix)
         {
             this.fieldNameX = fieldNamePrefix + SUFFIX_X;
@@ -79,20 +79,23 @@ namespace Lucene.Net.Spatial.Vector
             if (point != null)
                 return CreateIndexableFields(point);
 
-            throw new InvalidOperationException("Can only index Point, not " + shape);
+            throw new NotSupportedException("Can only index IPoint, not " + shape);
         }
 
+        /// <summary>
+        /// See <see cref="CreateIndexableFields(IShape)"/>
+        /// </summary>
         public Field[] CreateIndexableFields(IPoint point)
         {
             FieldType doubleFieldType = new FieldType(DoubleField.TYPE_NOT_STORED)
-                                            {
-                                                NumericPrecisionStep = precisionStep
-                                            };
-            var f = new Field[]
-                        {
-                            new DoubleField(fieldNameX, point.X, doubleFieldType),
-                            new DoubleField(fieldNameY, point.Y, doubleFieldType)
-                        };
+            {
+                NumericPrecisionStep = precisionStep
+            };
+            var f = new Field[2]
+            {
+                new DoubleField(fieldNameX, point.X, doubleFieldType),
+                new DoubleField(fieldNameY, point.Y, doubleFieldType)
+            };
             return f;
         }
 
@@ -101,22 +104,36 @@ namespace Lucene.Net.Spatial.Vector
             return new DistanceValueSource(this, queryPoint, multiplier);
         }
 
+        public override Filter MakeFilter(SpatialArgs args)
+        {
+            //unwrap the CSQ from makeQuery
+            ConstantScoreQuery csq = MakeQuery(args);
+            Filter filter = csq.Filter;
+            if (filter != null)
+                return filter;
+            else
+                return new QueryWrapperFilter(csq.Query);
+        }
+
         public override ConstantScoreQuery MakeQuery(SpatialArgs args)
         {
             if (!SpatialOperation.Is(args.Operation,
-                                     SpatialOperation.Intersects,
-                                     SpatialOperation.IsWithin))
+                SpatialOperation.Intersects,
+                SpatialOperation.IsWithin))
+            {
                 throw new UnsupportedSpatialOperation(args.Operation);
+            }
 
             IShape shape = args.Shape;
-            var bbox = shape as IRectangle;
-            if (bbox != null)
-                return new ConstantScoreQuery(new QueryWrapperFilter(MakeWithin(bbox)));
-
-            var circle = shape as ICircle;
-            if (circle != null)
+            if (shape is IRectangle)
             {
-                bbox = circle.BoundingBox;
+                var bbox = (IRectangle)shape;
+                return new ConstantScoreQuery(MakeWithin(bbox));
+            }
+            else if (shape is ICircle)
+            {
+                var circle = (ICircle)shape;
+                var bbox = circle.BoundingBox;
                 var vsf = new ValueSourceFilter(
                     new QueryWrapperFilter(MakeWithin(bbox)),
                     MakeDistanceValueSource(circle.Center),
@@ -124,29 +141,29 @@ namespace Lucene.Net.Spatial.Vector
                     circle.Radius);
                 return new ConstantScoreQuery(vsf);
             }
-
-            throw new NotSupportedException("Only Rectangles and Circles are currently supported, " +
+            
+            throw new NotSupportedException("Only IRectangles and ICircles are currently supported, " +
                                             "found [" + shape.GetType().Name + "]"); //TODO
         }
 
         //TODO this is basically old code that hasn't been verified well and should probably be removed
-        public Search.Query MakeQueryDistanceScore(SpatialArgs args)
+        public Query MakeQueryDistanceScore(SpatialArgs args)
         {
             // For starters, just limit the bbox
             var shape = args.Shape;
             if (!(shape is IRectangle || shape is ICircle))
-                throw new InvalidOperationException("Only Rectangles and Circles are currently supported, found ["
+                throw new NotSupportedException("Only Rectangles and Circles are currently supported, found ["
                     + shape.GetType().Name + "]");//TODO
 
             IRectangle bbox = shape.BoundingBox;
             if (bbox.CrossesDateLine)
             {
-                throw new InvalidOperationException("Crossing dateline not yet supported");
+                throw new NotSupportedException("Crossing dateline not yet supported");
             }
 
             ValueSource valueSource = null;
 
-            Search.Query spatial = null;
+            Query spatial = null;
             SpatialOperation op = args.Operation;
 
             if (SpatialOperation.Is(op,
@@ -160,9 +177,10 @@ namespace Lucene.Net.Spatial.Vector
               SpatialOperation.IsWithin))
             {
                 spatial = MakeWithin(bbox);
-                var circle = args.Shape as ICircle;
-                if (circle != null)
+                if (args.Shape is ICircle)
                 {
+                    var circle = (ICircle)args.Shape;
+
                     // Make the ValueSource
                     valueSource = MakeDistanceValueSource(shape.Center);
 
@@ -190,30 +208,17 @@ namespace Lucene.Net.Spatial.Vector
             {
                 valueSource = MakeDistanceValueSource(shape.Center);
             }
-            Search.Query spatialRankingQuery = new FunctionQuery(valueSource);
+            Query spatialRankingQuery = new FunctionQuery(valueSource);
             var bq = new BooleanQuery();
             bq.Add(spatial, BooleanClause.Occur.MUST);
             bq.Add(spatialRankingQuery, BooleanClause.Occur.MUST);
             return bq;
-
-        }
-
-        public override Filter MakeFilter(SpatialArgs args)
-        {
-            //unwrap the CSQ from makeQuery
-            ConstantScoreQuery csq = MakeQuery(args);
-            Filter filter = csq.Filter;
-            if (filter != null)
-                return filter;
-            else
-                return new QueryWrapperFilter(csq);
         }
 
         /// <summary>
         /// Constructs a query to retrieve documents that fully contain the input envelope.
         /// </summary>
-        /// <param name="bbox"></param>
-        private Search.Query MakeWithin(IRectangle bbox)
+        private Query MakeWithin(IRectangle bbox)
         {
             var bq = new BooleanQuery();
             const BooleanClause.Occur MUST = BooleanClause.Occur.MUST;
@@ -232,7 +237,7 @@ namespace Lucene.Net.Spatial.Vector
             return bq;
         }
 
-        private NumericRangeQuery<Double> RangeQuery(String fieldName, double? min, double? max)
+        private NumericRangeQuery<double> RangeQuery(string fieldName, double? min, double? max)
         {
             return NumericRangeQuery.NewDoubleRange(
                 fieldName,
@@ -246,14 +251,16 @@ namespace Lucene.Net.Spatial.Vector
         /// <summary>
         /// Constructs a query to retrieve documents that fully contain the input envelope.
         /// </summary>
-        /// <param name="bbox"></param>
-        private Search.Query MakeDisjoint(IRectangle bbox)
+        private Query MakeDisjoint(IRectangle bbox)
         {
             if (bbox.CrossesDateLine)
-                throw new InvalidOperationException("MakeDisjoint doesn't handle dateline cross");
-            Search.Query qX = RangeQuery(fieldNameX, bbox.MinX, bbox.MaxX);
-            Search.Query qY = RangeQuery(fieldNameY, bbox.MinY, bbox.MaxY);
-            var bq = new BooleanQuery { { qX, BooleanClause.Occur.MUST_NOT }, { qY, BooleanClause.Occur.MUST_NOT } };
+                throw new NotSupportedException("MakeDisjoint doesn't handle dateline cross");
+            Query qX = RangeQuery(fieldNameX, bbox.MinX, bbox.MaxX);
+            Query qY = RangeQuery(fieldNameY, bbox.MinY, bbox.MaxY);
+
+            var bq = new BooleanQuery();
+            bq.Add(qX, BooleanClause.Occur.MUST_NOT);
+            bq.Add(qY, BooleanClause.Occur.MUST_NOT);
             return bq;
         }
     }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/df3f64d7/src/Lucene.Net.Tests.Spatial/Prefix/SpatialOpRecursivePrefixTreeTest.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Prefix/SpatialOpRecursivePrefixTreeTest.cs b/src/Lucene.Net.Tests.Spatial/Prefix/SpatialOpRecursivePrefixTreeTest.cs
index d7d3fb7..d73667d 100644
--- a/src/Lucene.Net.Tests.Spatial/Prefix/SpatialOpRecursivePrefixTreeTest.cs
+++ b/src/Lucene.Net.Tests.Spatial/Prefix/SpatialOpRecursivePrefixTreeTest.cs
@@ -422,6 +422,7 @@ namespace Lucene.Net.Spatial.Prefix
             public ShapePair(IShape shape1, IShape shape2, bool containsThenWithin, SpatialContext ctx)
                         : base(Arrays.AsList(shape1, shape2), ctx)
             {
+                this.ctx = ctx;
 
                 this.shape1 = shape1;
                 this.shape2 = shape2;


[24/26] lucenenet git commit: Spatial: Refactor API members

Posted by ni...@apache.org.
Spatial: Refactor API members


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

Branch: refs/heads/master
Commit: b680810a379d9c3f6b27eb41b062322cdae46505
Parents: ad70ad8
Author: Shad Storhaug <sh...@shadstorhaug.com>
Authored: Fri Nov 18 02:37:54 2016 +0700
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Fri Nov 25 17:58:43 2016 +0700

----------------------------------------------------------------------
 .../Prefix/AbstractVisitingPrefixTreeFilter.cs  |  6 ++--
 .../Prefix/ContainsPrefixTreeFilter.cs          | 16 +++++------
 .../Prefix/IntersectsPrefixTreeFilter.cs        |  4 +--
 .../Prefix/PointPrefixTreeFieldCacheProvider.cs |  4 +--
 .../Prefix/PrefixTreeStrategy.cs                |  2 +-
 src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs      | 30 ++++++++++----------
 .../Prefix/Tree/GeohashPrefixTree.cs            | 19 +++++++------
 .../Prefix/Tree/QuadPrefixTree.cs               | 25 +++++++++-------
 .../Prefix/Tree/SpatialPrefixTree.cs            |  8 +++---
 .../Prefix/Tree/SpatialPrefixTreeFactory.cs     | 16 +++++------
 .../Prefix/WithinPrefixTreeFilter.cs            |  6 ++--
 src/Lucene.Net.Spatial/Query/SpatialArgs.cs     |  2 +-
 .../Query/SpatialOperation.cs                   |  6 ++--
 .../Prefix/SpatialOpRecursivePrefixTreeTest.cs  |  2 +-
 .../Prefix/Tree/SpatialPrefixTreeTest.cs        |  6 ++--
 15 files changed, 79 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/b680810a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
index 1286158..6afee2c 100644
--- a/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/AbstractVisitingPrefixTreeFilter.cs
@@ -280,7 +280,7 @@ namespace Lucene.Net.Spatial.Prefix
                     // then add all of those docs
                     Debug.Assert(StringHelper.StartsWith(thisTerm, curVNodeTerm));//TODO refactor to use method on curVNode.cell
                     scanCell = outerInstance.grid.GetCell(thisTerm.Bytes, thisTerm.Offset, thisTerm.Length, scanCell);
-                    if (scanCell.Level == cell.Level && scanCell.IsLeaf())
+                    if (scanCell.Level == cell.Level && scanCell.IsLeaf)
                     {
                         VisitLeaf(scanCell);
                         //advance
@@ -356,12 +356,12 @@ namespace Lucene.Net.Spatial.Prefix
                     int termLevel = scanCell.Level;
                     if (termLevel < scanDetailLevel)
                     {
-                        if (scanCell.IsLeaf())
+                        if (scanCell.IsLeaf)
                             VisitScanned(scanCell);
                     }
                     else if (termLevel == scanDetailLevel)
                     {
-                        if (!scanCell.IsLeaf())//LUCENE-5529
+                        if (!scanCell.IsLeaf)//LUCENE-5529
                             VisitScanned(scanCell);
                     }
                 }//term loop

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/b680810a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
index ea10507..692722e 100644
--- a/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/ContainsPrefixTreeFilter.cs
@@ -99,10 +99,10 @@ namespace Lucene.Net.Spatial.Prefix
                 SmallDocSet combinedSubResults = null;
                 //   Optimization: use null subCellsFilter when we know cell is within the query shape.
                 IShape subCellsFilter = outerInstance.queryShape;
-                if (cell.Level != 0 && ((cell.GetShapeRel() == SpatialRelation.NULL_VALUE || cell.GetShapeRel() == SpatialRelation.WITHIN)))
+                if (cell.Level != 0 && ((cell.ShapeRel == SpatialRelation.NULL_VALUE || cell.ShapeRel == SpatialRelation.WITHIN)))
                 {
                     subCellsFilter = null;
-                    Debug.Assert(cell.GetShape().Relate(outerInstance.queryShape) == SpatialRelation.WITHIN);
+                    Debug.Assert(cell.Shape.Relate(outerInstance.queryShape) == SpatialRelation.WITHIN);
                 }
                 ICollection<Cell> subCells = cell.GetSubCells(subCellsFilter);
                 foreach (Cell subCell in subCells)
@@ -116,7 +116,7 @@ namespace Lucene.Net.Spatial.Prefix
                         combinedSubResults = GetDocs(subCell, acceptContains);
                     }
                     else if (!outerInstance.multiOverlappingIndexedShapes && 
-                        subCell.GetShapeRel() == SpatialRelation.WITHIN)
+                        subCell.ShapeRel == SpatialRelation.WITHIN)
                     {
                         combinedSubResults = GetLeafDocs(subCell, acceptContains); //recursion
                     }
@@ -178,7 +178,7 @@ namespace Lucene.Net.Spatial.Prefix
                     return null;
                 }
                 nextCell = outerInstance.grid.GetCell(nextTerm.Bytes, nextTerm.Offset, nextTerm.Length, this.nextCell);
-                if (nextCell.Level == leafCell.Level && nextCell.IsLeaf())
+                if (nextCell.Level == leafCell.Level && nextCell.IsLeaf)
                 {
                     return CollectDocs(acceptContains);
                 }
@@ -248,9 +248,9 @@ namespace Lucene.Net.Spatial.Prefix
 
             /// <summary>Number of docids.</summary>
             /// <remarks>Number of docids.</remarks>
-            public virtual int Size()
+            public virtual int Size
             {
-                return intSet.Size();
+                get { return intSet.Size(); }
             }
 
             /// <summary>NOTE: modifies and returns either "this" or "other"</summary>
@@ -284,7 +284,7 @@ namespace Lucene.Net.Spatial.Prefix
             {
                 //if the # of docids is super small, return null since iteration is going
                 // to be faster
-                return Size() > 4 ? this : null;
+                return Size > 4 ? this : null;
             }
 
             private sealed class _DocIdSetIterator_225 : DocIdSetIterator
@@ -336,7 +336,7 @@ namespace Lucene.Net.Spatial.Prefix
 
             public override DocIdSetIterator GetIterator()
             {
-                if (Size() == 0)
+                if (Size == 0)
                 {
                     return null;
                 }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/b680810a/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
index f5c288d..e9916e1 100644
--- a/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/IntersectsPrefixTreeFilter.cs
@@ -87,7 +87,7 @@ namespace Lucene.Net.Spatial.Prefix
 
             protected internal override bool Visit(Cell cell)
             {
-                if (cell.GetShapeRel() == SpatialRelation.WITHIN || cell.Level == outerInstance.detailLevel)
+                if (cell.ShapeRel == SpatialRelation.WITHIN || cell.Level == outerInstance.detailLevel)
                 {
                     CollectDocs(results);
                     return false;
@@ -102,7 +102,7 @@ namespace Lucene.Net.Spatial.Prefix
 
             protected internal override void VisitScanned(Cell cell)
             {
-                if (outerInstance.queryShape.Relate(cell.GetShape()).Intersects())
+                if (outerInstance.queryShape.Relate(cell.Shape).Intersects())
                 {
                     CollectDocs(results);
                 }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/b680810a/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs b/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
index f89da09..d4a2ef3 100644
--- a/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
+++ b/src/Lucene.Net.Spatial/Prefix/PointPrefixTreeFieldCacheProvider.cs
@@ -47,9 +47,9 @@ namespace Lucene.Net.Spatial.Prefix
         protected internal override IPoint ReadShape(BytesRef term)
         {
             scanCell = grid.GetCell(term.Bytes, term.Offset, term.Length, scanCell);
-            if (scanCell.Level == grid.MaxLevels && !scanCell.IsLeaf())
+            if (scanCell.Level == grid.MaxLevels && !scanCell.IsLeaf)
             {
-                return scanCell.GetCenter();
+                return scanCell.Center;
             }
             return null;
         }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/b680810a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs b/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
index 5176771..5bd7605 100644
--- a/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
+++ b/src/Lucene.Net.Spatial/Prefix/PrefixTreeStrategy.cs
@@ -204,7 +204,7 @@ namespace Lucene.Net.Spatial.Prefix
                     Cell cell = iter.Current;
                     string token = cell.TokenString;
                     termAtt.Append(token);
-                    if (cell.IsLeaf())
+                    if (cell.IsLeaf)
                     {
                         nextTokenStringNeedingLeaf = token;
                     }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/b680810a/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs b/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
index 57a04ca..1e5e488 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/Cell.cs
@@ -71,7 +71,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         protected internal Cell(SpatialPrefixTree outerInstance, string token)
         {
             // LUCENENET specific - set the outer instance here
-            // because overrides of GetShape() may require it
+            // because overrides of Shape may require it
             this.outerInstance = outerInstance;
 
             //NOTE: must sort before letters & numbers
@@ -84,14 +84,14 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             }
             if (Level == 0)
             {
-                GetShape();//ensure any lazy instantiation completes to make this threadsafe
+                var x = Shape;//ensure any lazy instantiation completes to make this threadsafe
             }
         }
 
         protected internal Cell(SpatialPrefixTree outerInstance, byte[] bytes, int off, int len)
         {
             // LUCENENET specific - set the outer instance here
-            // because overrides of GetShape() may require it
+            // because overrides of Shape may require it
             this.outerInstance = outerInstance;
 
             //ensure any lazy instantiation completes to make this threadsafe
@@ -126,9 +126,9 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             }
         }
 
-        public virtual SpatialRelation GetShapeRel()
+        public virtual SpatialRelation ShapeRel
         {
-            return shapeRel;
+            get { return shapeRel; }
         }
 
         /// <summary>For points, this is always false.</summary>
@@ -136,9 +136,9 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// For points, this is always false.  Otherwise this is true if there are no
         /// further cells with this prefix for the shape (always true at maxLevels).
         /// </remarks>
-        public virtual bool IsLeaf()
+        public virtual bool IsLeaf
         {
-            return leaf;
+            get { return leaf; }
         }
 
         /// <summary>Note: not supported at level 0.</summary>
@@ -152,7 +152,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// <summary>
         /// Note: doesn't contain a trailing leaf byte.
         /// </summary>
-        public virtual String TokenString
+        public virtual string TokenString
         {
             get
             {
@@ -202,7 +202,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// 	</see>
         /// . The returned cells
         /// should have
-        /// <see cref="GetShapeRel()">GetShapeRel()</see>
+        /// <see cref="ShapeRel">ShapeRel</see>
         /// set to their relation with
         /// <code>shapeFilter</code>
         /// . In addition,
@@ -236,7 +236,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             IList<Cell> copy = new List<Cell>(cells.Count);
             foreach (Cell cell in cells)
             {
-                SpatialRelation rel = cell.GetShape().Relate(shapeFilter);
+                SpatialRelation rel = cell.Shape.Relate(shapeFilter);
                 if (rel == SpatialRelation.DISJOINT)
                 {
                     continue;
@@ -277,13 +277,13 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         /// <see cref="GetSubCells()">GetSubCells()</see>
         /// .size() -- usually a constant. Should be &gt;=2
         /// </summary>
-        public abstract int GetSubCellsSize();
+        public abstract int SubCellsSize { get; }
 
-        public abstract IShape GetShape();
+        public abstract IShape Shape { get; }
 
-        public virtual IPoint GetCenter()
+        public virtual IPoint Center
         {
-            return GetShape().Center;
+            get { return Shape.Center; }
         }
 
         #region IComparable<Cell> Members
@@ -310,7 +310,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
 
         public override string ToString()
         {
-            return TokenString + (IsLeaf() ? ((char)LEAF_BYTE).ToString() : string.Empty);
+            return TokenString + (IsLeaf ? ((char)LEAF_BYTE).ToString() : string.Empty);
         }
 
         #endregion

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/b680810a/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
index a11f47e..5dce614 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/GeohashPrefixTree.cs
@@ -138,9 +138,9 @@ namespace Lucene.Net.Spatial.Prefix.Tree
                 return cells;
             }
 
-            public override int GetSubCellsSize()
+            public override int SubCellsSize
             {
-                return 32;//8x4
+                get { return 32; }//8x4
             }
 
             public override Cell GetSubCell(IPoint p)
@@ -150,18 +150,21 @@ namespace Lucene.Net.Spatial.Prefix.Tree
 
             private IShape shape;//cache
 
-            public override IShape GetShape()
+            public override IShape Shape
             {
-                if (shape == null)
+                get
                 {
-                    shape = GeohashUtils.DecodeBoundary(Geohash, outerInstance.ctx);
+                    if (shape == null)
+                    {
+                        shape = GeohashUtils.DecodeBoundary(Geohash, outerInstance.ctx);
+                    }
+                    return shape;
                 }
-                return shape;
             }
 
-            public override IPoint GetCenter()
+            public override IPoint Center
             {
-                return GeohashUtils.Decode(Geohash, outerInstance.ctx);
+                get { return GeohashUtils.Decode(Geohash, outerInstance.ctx); }
             }
 
             private string Geohash

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/b680810a/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
index 1222d27..1e90dc2 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/QuadPrefixTree.cs
@@ -46,21 +46,21 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         {
             protected internal override int GetLevelForDistance(double degrees)
             {
-                var grid = new QuadPrefixTree(ctx, MaxLevelsPossible);
+                var grid = new QuadPrefixTree(ctx, MAX_LEVELS_POSSIBLE);
                 return grid.GetLevelForDistance(degrees);
             }
 
             protected internal override SpatialPrefixTree NewSPT()
             {
-                return new QuadPrefixTree(ctx, maxLevels.HasValue ? maxLevels.Value : MaxLevelsPossible);
+                return new QuadPrefixTree(ctx, maxLevels.HasValue ? maxLevels.Value : MAX_LEVELS_POSSIBLE);
             }
         }
 
         #endregion
 
-        public const int MaxLevelsPossible = 50;//not really sure how big this should be
+        public const int MAX_LEVELS_POSSIBLE = 50;//not really sure how big this should be
 
-        public const int DefaultMaxLevels = 12;
+        public const int DEFAULT_MAX_LEVELS = 12;
         private readonly double xmin;
         private readonly double xmax;
         private readonly double ymin;
@@ -108,7 +108,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         }
 
         public QuadPrefixTree(SpatialContext ctx)
-            : this(ctx, DefaultMaxLevels)
+            : this(ctx, DEFAULT_MAX_LEVELS)
         {
         }
 
@@ -270,9 +270,9 @@ namespace Lucene.Net.Spatial.Prefix.Tree
                 return cells;
             }
 
-            public override int GetSubCellsSize()
+            public override int SubCellsSize
             {
-                return 4;
+                get { return 4; }
             }
 
             public override Cell GetSubCell(IPoint p)
@@ -282,13 +282,16 @@ namespace Lucene.Net.Spatial.Prefix.Tree
 
             private IShape shape; //cache
 
-            public override IShape GetShape()
+            public override IShape Shape
             {
-                if (shape == null)
+                get
                 {
-                    shape = MakeShape();
+                    if (shape == null)
+                    {
+                        shape = MakeShape();
+                    }
+                    return shape;
                 }
-                return shape;
             }
 
             private IRectangle MakeShape()

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/b680810a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
index 9e5f2a3..3fdac52 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTree.cs
@@ -101,7 +101,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             }
             //TODO cache for each level
             Cell cell = GetCell(ctx.WorldBounds.Center, level);
-            IRectangle bbox = cell.GetShape().BoundingBox;
+            IRectangle bbox = cell.Shape.BoundingBox;
             double width = bbox.Width;
             double height = bbox.Height;
             //Use standard cartesian hypotenuse. For geospatial, this answer is larger
@@ -222,7 +222,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             {
                 cell.SetLeaf();//FYI might already be a leaf
             }
-            if (cell.IsLeaf())
+            if (cell.IsLeaf)
             {
                 result.Add(cell);
                 return true;
@@ -242,7 +242,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
                 }
             }
             //can we simplify?
-            if (simplify && leaves == cell.GetSubCellsSize() && cell.Level != 0)
+            if (simplify && leaves == cell.SubCellsSize && cell.Level != 0)
             {
                 //Optimization: substitute the parent as a leaf instead of adding all
                 // children as leaves
@@ -307,7 +307,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             foreach (Cell cell in cells)
             {
                 string token = cell.TokenString;
-                if (cell.IsLeaf())
+                if (cell.IsLeaf)
                 {
                     tokens.Add(token + (char)Cell.LEAF_BYTE);
                 }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/b680810a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
index a1905ca..34984b4 100644
--- a/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
+++ b/src/Lucene.Net.Spatial/Prefix/Tree/SpatialPrefixTreeFactory.cs
@@ -32,10 +32,10 @@ namespace Lucene.Net.Spatial.Prefix.Tree
     /// <lucene.experimental></lucene.experimental>
     public abstract class SpatialPrefixTreeFactory
     {
-        private const double DefaultGeoMaxDetailKm = 0.001;//1m
-        public const string PrefixTree = "prefixTree";
-        public const string MaxLevels = "maxLevels";
-        public const string MaxDistErr = "maxDistErr";
+        private const double DEFAULT_GEO_MAX_DETAIL_KM = 0.001;//1m
+        public const string PREFIX_TREE = "prefixTree";
+        public const string MAX_LEVELS = "maxLevels";
+        public const string MAX_DIST_ERR = "maxDistErr";
 
         protected internal IDictionary<string, string> args;
         protected internal SpatialContext ctx;
@@ -51,7 +51,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         {
             SpatialPrefixTreeFactory instance;
             string cname;
-            if (!args.TryGetValue(PrefixTree, out cname))
+            if (!args.TryGetValue(PREFIX_TREE, out cname))
             {
                 cname = ctx.IsGeo ? "geohash" : "quad";
             }
@@ -89,21 +89,21 @@ namespace Lucene.Net.Spatial.Prefix.Tree
         protected internal virtual void InitMaxLevels()
         {
             string mlStr;
-            if (args.TryGetValue(MaxLevels, out mlStr))
+            if (args.TryGetValue(MAX_LEVELS, out mlStr))
             {
                 maxLevels = int.Parse(mlStr, CultureInfo.InvariantCulture);
                 return;
             }
             double degrees;
             string maxDetailDistStr;
-            if (!args.TryGetValue(MaxDistErr, out maxDetailDistStr))
+            if (!args.TryGetValue(MAX_DIST_ERR, out maxDetailDistStr))
             {
                 if (!ctx.IsGeo)
                 {
                     return;
                 }
                 //let default to max
-                degrees = DistanceUtils.Dist2Degrees(DefaultGeoMaxDetailKm, DistanceUtils.EARTH_MEAN_RADIUS_KM);
+                degrees = DistanceUtils.Dist2Degrees(DEFAULT_GEO_MAX_DETAIL_KM, DistanceUtils.EARTH_MEAN_RADIUS_KM);
             }
             else
             {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/b680810a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
index f78dfb9..fc9f5b3 100644
--- a/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
+++ b/src/Lucene.Net.Spatial/Prefix/WithinPrefixTreeFilter.cs
@@ -185,7 +185,7 @@ namespace Lucene.Net.Spatial.Prefix
             {
                 //cell.relate is based on the bufferedQueryShape; we need to examine what
                 // the relation is against the queryShape
-                visitRelation = cell.GetShape().Relate(outerInstance.queryShape);
+                visitRelation = cell.Shape.Relate(outerInstance.queryShape);
                 if (visitRelation == SpatialRelation.WITHIN)
                 {
                     CollectDocs(inside);
@@ -209,7 +209,7 @@ namespace Lucene.Net.Spatial.Prefix
             {
                 //visitRelation is declared as a field, populated by visit() so we don't recompute it
                 Debug.Assert(outerInstance.detailLevel != cell.Level);
-                Debug.Assert(visitRelation == cell.GetShape().Relate(outerInstance.queryShape));
+                Debug.Assert(visitRelation == cell.Shape.Relate(outerInstance.queryShape));
                 if (AllCellsIntersectQuery(cell, visitRelation))
                 {
                     CollectDocs(inside);
@@ -232,7 +232,7 @@ namespace Lucene.Net.Spatial.Prefix
             {
                 if (relate == SpatialRelation.NULL_VALUE)
                 {
-                    relate = cell.GetShape().Relate(outerInstance.queryShape);
+                    relate = cell.Shape.Relate(outerInstance.queryShape);
                 }
                 if (cell.Level == outerInstance.detailLevel)
                 {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/b680810a/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialArgs.cs b/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
index 15dc0f1..7c49f5d 100644
--- a/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
+++ b/src/Lucene.Net.Spatial/Query/SpatialArgs.cs
@@ -89,7 +89,7 @@ namespace Lucene.Net.Spatial.Queries
         /// </summary>
         public virtual void Validate()
         {
-            if (Operation.IsTargetNeedsArea && !Shape.HasArea)
+            if (Operation.TargetNeedsArea && !Shape.HasArea)
             {
                 throw new ArgumentException(Operation + " only supports geometry with area");
             }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/b680810a/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Spatial/Query/SpatialOperation.cs b/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
index ac1c9f3..a05703e 100644
--- a/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
+++ b/src/Lucene.Net.Spatial/Query/SpatialOperation.cs
@@ -208,17 +208,17 @@ namespace Lucene.Net.Spatial.Queries
 
         // ================================================= Getters / Setters =============================================
 
-        public virtual bool IsScoreIsMeaningful
+        public virtual bool IsScoreMeaningful
         {
             get { return scoreIsMeaningful; }
         }
 
-        public virtual bool IsSourceNeedsArea
+        public virtual bool SourceNeedsArea
         {
             get { return sourceNeedsArea; }
         }
 
-        public virtual bool IsTargetNeedsArea
+        public virtual bool TargetNeedsArea
         {
             get { return targetNeedsArea; }
         }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/b680810a/src/Lucene.Net.Tests.Spatial/Prefix/SpatialOpRecursivePrefixTreeTest.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Prefix/SpatialOpRecursivePrefixTreeTest.cs b/src/Lucene.Net.Tests.Spatial/Prefix/SpatialOpRecursivePrefixTreeTest.cs
index d73667d..d53f4a8 100644
--- a/src/Lucene.Net.Tests.Spatial/Prefix/SpatialOpRecursivePrefixTreeTest.cs
+++ b/src/Lucene.Net.Tests.Spatial/Prefix/SpatialOpRecursivePrefixTreeTest.cs
@@ -400,7 +400,7 @@ namespace Lucene.Net.Spatial.Prefix
             List<IShape> cellShapes = new List<IShape>(cells.size());
             foreach (Cell cell in cells)
             {
-                cellShapes.Add(cell.GetShape());
+                cellShapes.Add(cell.Shape);
             }
             return new ShapeCollection(cellShapes, ctx).BoundingBox;
         }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/b680810a/src/Lucene.Net.Tests.Spatial/Prefix/Tree/SpatialPrefixTreeTest.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Spatial/Prefix/Tree/SpatialPrefixTreeTest.cs b/src/Lucene.Net.Tests.Spatial/Prefix/Tree/SpatialPrefixTreeTest.cs
index 581b2f6..5dcc203 100644
--- a/src/Lucene.Net.Tests.Spatial/Prefix/Tree/SpatialPrefixTreeTest.cs
+++ b/src/Lucene.Net.Tests.Spatial/Prefix/Tree/SpatialPrefixTreeTest.cs
@@ -45,7 +45,7 @@ namespace Lucene.Net.Spatial.Prefix.Tree
             Cell prevC = null;
             Cell c = trie.WorldCell;
             assertEquals(0, c.Level);
-            assertEquals(ctx.WorldBounds, c.GetShape());
+            assertEquals(ctx.WorldBounds, c.Shape);
             while (c.Level < trie.MaxLevels)
             {
                 prevC = c;
@@ -56,8 +56,8 @@ namespace Lucene.Net.Spatial.Prefix.Tree
                 //c = c.GetSubCells().GetEnumerator().next();//TODO random which one?
 
                 assertEquals(prevC.Level + 1, c.Level);
-                IRectangle prevNShape = (IRectangle)prevC.GetShape();
-                IShape s = c.GetShape();
+                IRectangle prevNShape = (IRectangle)prevC.Shape;
+                IShape s = c.Shape;
                 IRectangle sbox = s.BoundingBox;
                 assertTrue(prevNShape.Width > sbox.Width);
                 assertTrue(prevNShape.Height > sbox.Height);