You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by nk...@apache.org on 2016/02/29 23:20:10 UTC

[24/29] lucene-solr git commit: LUCENE-7015: Refactor spatial module to spatial-extras

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/CachingDoubleValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/CachingDoubleValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/CachingDoubleValueSource.java
new file mode 100644
index 0000000..73d25ca
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/CachingDoubleValueSource.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.util;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Caches the doubleVal of another value source in a HashMap
+ * so that it is computed only once.
+ * @lucene.internal
+ */
+public class CachingDoubleValueSource extends ValueSource {
+
+  final ValueSource source;
+  final Map<Integer, Double> cache;
+
+  public CachingDoubleValueSource( ValueSource source )
+  {
+    this.source = source;
+    cache = new HashMap<>();
+  }
+
+  @Override
+  public String description() {
+    return "Cached["+source.description()+"]";
+  }
+
+  @Override
+  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
+    final int base = readerContext.docBase;
+    final FunctionValues vals = source.getValues(context,readerContext);
+    return new FunctionValues() {
+
+      @Override
+      public double doubleVal(int doc) {
+        Integer key = Integer.valueOf( base+doc );
+        Double v = cache.get( key );
+        if( v == null ) {
+          v = Double.valueOf( vals.doubleVal(doc) );
+          cache.put( key, v );
+        }
+        return v.doubleValue();
+      }
+
+      @Override
+      public float floatVal(int doc) {
+        return (float)doubleVal(doc);
+      }
+
+      @Override
+      public String toString(int doc) {
+        return doubleVal(doc)+"";
+      }
+    };
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    CachingDoubleValueSource that = (CachingDoubleValueSource) o;
+
+    if (source != null ? !source.equals(that.source) : that.source != null) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    return source != null ? source.hashCode() : 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java
new file mode 100644
index 0000000..57cad87
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.util;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.IndexSearcher;
+
+import com.spatial4j.core.context.SpatialContext;
+import com.spatial4j.core.distance.DistanceCalculator;
+import com.spatial4j.core.shape.Point;
+import com.spatial4j.core.shape.Shape;
+
+/**
+ * The distance from a provided Point to a Point retrieved from a ValueSource via
+ * {@link org.apache.lucene.queries.function.FunctionValues#objectVal(int)}. The distance
+ * is calculated via a {@link com.spatial4j.core.distance.DistanceCalculator}.
+ *
+ * @lucene.experimental
+ */
+public class DistanceToShapeValueSource extends ValueSource {
+  private final ValueSource shapeValueSource;
+  private final Point queryPoint;
+  private final double multiplier;
+  private final DistanceCalculator distCalc;
+
+  //TODO if FunctionValues returns NaN; will things be ok?
+  private final double nullValue;//computed
+
+  public DistanceToShapeValueSource(ValueSource shapeValueSource, Point queryPoint,
+                                    double multiplier, SpatialContext ctx) {
+    this.shapeValueSource = shapeValueSource;
+    this.queryPoint = queryPoint;
+    this.multiplier = multiplier;
+    this.distCalc = ctx.getDistCalc();
+    this.nullValue =
+        (ctx.isGeo() ? 180 * multiplier : Double.MAX_VALUE);
+  }
+
+  @Override
+  public String description() {
+    return "distance(" + queryPoint + " to " + shapeValueSource.description() + ")*" + multiplier + ")";
+  }
+
+  @Override
+  public void createWeight(Map context, IndexSearcher searcher) throws IOException {
+    shapeValueSource.createWeight(context, searcher);
+  }
+
+  @Override
+  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
+    final FunctionValues shapeValues = shapeValueSource.getValues(context, readerContext);
+
+    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);
+        List<Explanation> details = new ArrayList<>(Arrays.asList(exp.getDetails()));
+        details.add(shapeValues.explain(doc));
+        return Explanation.match(exp.getValue(), exp.getDescription(), details);
+      }
+    };
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    DistanceToShapeValueSource that = (DistanceToShapeValueSource) o;
+
+    if (!queryPoint.equals(that.queryPoint)) return false;
+    if (Double.compare(that.multiplier, multiplier) != 0) return false;
+    if (!shapeValueSource.equals(that.shapeValueSource)) return false;
+    if (!distCalc.equals(that.distCalc)) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result;
+    long temp;
+    result = shapeValueSource.hashCode();
+    result = 31 * result + queryPoint.hashCode();
+    temp = Double.doubleToLongBits(multiplier);
+    result = 31 * result + (int) (temp ^ (temp >>> 32));
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeAreaValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeAreaValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeAreaValueSource.java
new file mode 100644
index 0000000..dd391d1
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeAreaValueSource.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.util;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import com.spatial4j.core.context.SpatialContext;
+import com.spatial4j.core.shape.Shape;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.IndexSearcher;
+
+/**
+ * The area of a Shape retrieved from a ValueSource via
+ * {@link org.apache.lucene.queries.function.FunctionValues#objectVal(int)}.
+ *
+ * @see Shape#getArea(com.spatial4j.core.context.SpatialContext)
+ *
+ * @lucene.experimental
+ */
+public class ShapeAreaValueSource extends ValueSource {
+  private final ValueSource shapeValueSource;
+  private final SpatialContext ctx;//not part of identity; should be associated with shapeValueSource indirectly
+  private final boolean geoArea;
+  private double multiplier;
+
+  public ShapeAreaValueSource(ValueSource shapeValueSource, SpatialContext ctx, boolean geoArea, double multiplier) {
+    this.shapeValueSource = shapeValueSource;
+    this.ctx = ctx;
+    this.geoArea = geoArea;
+    this.multiplier = multiplier;
+  }
+
+  @Override
+  public String description() {
+    return "area(" + shapeValueSource.description() + ",geo=" + geoArea + ")";
+  }
+
+  @Override
+  public void createWeight(Map context, IndexSearcher searcher) throws IOException {
+    shapeValueSource.createWeight(context, searcher);
+  }
+
+  @Override
+  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
+    final FunctionValues shapeValues = shapeValueSource.getValues(context, readerContext);
+
+    return new DoubleDocValues(this) {
+      @Override
+      public double doubleVal(int doc) {
+        Shape shape = (Shape) shapeValues.objectVal(doc);
+        if (shape == null || shape.isEmpty())
+          return 0;//or NaN?
+        //This part of Spatial4j API is kinda weird. Passing null means 2D area, otherwise geo
+        //   assuming ctx.isGeo()
+        return shape.getArea( geoArea ? ctx : null ) * multiplier;
+      }
+
+      @Override
+      public boolean exists(int doc) {
+        return shapeValues.exists(doc);
+      }
+
+      @Override
+      public Explanation explain(int doc) {
+        Explanation exp = super.explain(doc);
+        List<Explanation> details = new ArrayList<>(Arrays.asList(exp.getDetails()));
+        details.add(shapeValues.explain(doc));
+        return Explanation.match(exp.getValue(), exp.getDescription(), details);
+      }
+    };
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    ShapeAreaValueSource that = (ShapeAreaValueSource) o;
+
+    if (geoArea != that.geoArea) return false;
+    if (!shapeValueSource.equals(that.shapeValueSource)) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = shapeValueSource.hashCode();
+    result = 31 * result + (geoArea ? 1 : 0);
+    return result;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCache.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCache.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCache.java
new file mode 100644
index 0000000..480369b
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCache.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.util;
+
+import com.spatial4j.core.shape.Shape;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Bounded Cache of Shapes associated with docIds.  Note, multiple Shapes can be
+ * associated with a given docId.
+ * <p>
+ * 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.
+ *
+ * @lucene.internal
+ */
+public class ShapeFieldCache<T extends Shape> {
+  private final List<T>[] cache;
+  public final int defaultLength;
+
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  public ShapeFieldCache( int length, int defaultLength ) {
+    cache = new List[length];
+    this.defaultLength= defaultLength;
+  }
+
+  public void add( int docid, T s ) {
+    List<T> list = cache[docid];
+    if( list == null ) {
+      list = cache[docid] = new ArrayList<>(defaultLength);
+    }
+    list.add( s );
+  }
+
+  public List<T> getShapes( int docid ) {
+    return cache[docid];
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java
new file mode 100644
index 0000000..e4cb146
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.util;
+
+import com.spatial4j.core.context.SpatialContext;
+import com.spatial4j.core.distance.DistanceCalculator;
+import com.spatial4j.core.shape.Point;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * An implementation of the Lucene ValueSource that returns the spatial distance
+ * between an input point and a document's points in
+ * {@link ShapeFieldCacheProvider}. The shortest distance is returned if a
+ * document has more than one point.
+ *
+ * @lucene.internal
+ */
+public class ShapeFieldCacheDistanceValueSource extends ValueSource {
+
+  private final SpatialContext ctx;
+  private final Point from;
+  private final ShapeFieldCacheProvider<Point> provider;
+  private final double multiplier;
+
+  public ShapeFieldCacheDistanceValueSource(SpatialContext ctx,
+      ShapeFieldCacheProvider<Point> provider, Point from, double multiplier) {
+    this.ctx = ctx;
+    this.from = from;
+    this.provider = provider;
+    this.multiplier = multiplier;
+  }
+
+  @Override
+  public String description() {
+    return getClass().getSimpleName()+"("+provider+", "+from+")";
+  }
+
+  @Override
+  public FunctionValues getValues(Map context, final LeafReaderContext readerContext) throws IOException {
+    return new FunctionValues() {
+      private final ShapeFieldCache<Point> cache =
+          provider.getCache(readerContext.reader());
+      private final Point from = ShapeFieldCacheDistanceValueSource.this.from;
+      private final DistanceCalculator calculator = ctx.getDistCalc();
+      private final double nullValue = (ctx.isGeo() ? 180 * multiplier : Double.MAX_VALUE);
+
+      @Override
+      public float floatVal(int doc) {
+        return (float) doubleVal(doc);
+      }
+
+      @Override
+      public double doubleVal(int doc) {
+
+        List<Point> vals = cache.getShapes( doc );
+        if( vals != null ) {
+          double v = calculator.distance(from, vals.get(0));
+          for( int i=1; i<vals.size(); i++ ) {
+            v = Math.min(v, calculator.distance(from, vals.get(i)));
+          }
+          return v * multiplier;
+        }
+        return nullValue;
+      }
+
+      @Override
+      public String toString(int doc) {
+        return description() + "=" + floatVal(doc);
+      }
+    };
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    ShapeFieldCacheDistanceValueSource that = (ShapeFieldCacheDistanceValueSource) o;
+
+    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;
+  }
+
+  @Override
+  public int hashCode() {
+    return from.hashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheProvider.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheProvider.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheProvider.java
new file mode 100644
index 0000000..04c52f7
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheProvider.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.util;
+
+import com.spatial4j.core.shape.Shape;
+import org.apache.lucene.index.*;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BytesRef;
+
+import java.io.IOException;
+import java.util.WeakHashMap;
+import java.util.logging.Logger;
+
+/**
+ * Provides access to a {@link ShapeFieldCache} for a given {@link org.apache.lucene.index.LeafReader}.
+ *
+ * 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.
+ *
+ * @lucene.internal
+ */
+public abstract class ShapeFieldCacheProvider<T extends Shape> {
+  private Logger log = Logger.getLogger(getClass().getName());
+
+  // it may be a List<T> or T
+  WeakHashMap<IndexReader, ShapeFieldCache<T>> sidx = new WeakHashMap<>();
+
+  protected final int defaultSize;
+  protected final String shapeField;
+
+  public ShapeFieldCacheProvider(String shapeField, int defaultSize) {
+    this.shapeField = shapeField;
+    this.defaultSize = defaultSize;
+  }
+
+  protected abstract T readShape( BytesRef term );
+
+  public synchronized ShapeFieldCache<T> getCache(LeafReader reader) throws IOException {
+    ShapeFieldCache<T> idx = sidx.get(reader);
+    if (idx != null) {
+      return idx;
+    }
+    long startTime = System.currentTimeMillis();
+
+    log.fine("Building Cache [" + reader.maxDoc() + "]");
+    idx = new ShapeFieldCache<>(reader.maxDoc(),defaultSize);
+    int count = 0;
+    PostingsEnum docs = null;
+    Terms terms = reader.terms(shapeField);
+    if (terms != null) {
+      TermsEnum te = terms.iterator();
+      BytesRef term = te.next();
+      while (term != null) {
+        T shape = readShape(term);
+        if( shape != null ) {
+          docs = te.postings(docs, PostingsEnum.NONE);
+          Integer docid = docs.nextDoc();
+          while (docid != DocIdSetIterator.NO_MORE_DOCS) {
+            idx.add( docid, shape );
+            docid = docs.nextDoc();
+            count++;
+          }
+        }
+        term = te.next();
+      }
+    }
+    sidx.put(reader, idx);
+    long elapsed = System.currentTimeMillis() - startTime;
+    log.fine("Cached: [" + count + " in " + elapsed + "ms] " + idx);
+    return idx;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapePredicateValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapePredicateValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapePredicateValueSource.java
new file mode 100644
index 0000000..b1dfaaa
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapePredicateValueSource.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.util;
+
+import com.spatial4j.core.shape.Shape;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.docvalues.BoolDocValues;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.spatial.query.SpatialOperation;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A boolean ValueSource that compares a shape from a provided ValueSource with a given Shape and sees
+ * if it matches a given {@link SpatialOperation} (the predicate).
+ *
+ * @lucene.experimental
+ */
+public class ShapePredicateValueSource extends ValueSource {
+  private final ValueSource shapeValuesource;//the left hand side
+  private final SpatialOperation op;
+  private final Shape queryShape;//the right hand side (constant)
+
+  /**
+   *
+   * @param shapeValuesource Must yield {@link Shape} instances from its objectVal(doc). If null
+   *                         then the result is false. This is the left-hand (indexed) side.
+   * @param op the predicate
+   * @param queryShape The shape on the right-hand (query) side.
+   */
+  public ShapePredicateValueSource(ValueSource shapeValuesource, SpatialOperation op, Shape queryShape) {
+    this.shapeValuesource = shapeValuesource;
+    this.op = op;
+    this.queryShape = queryShape;
+  }
+
+  @Override
+  public String description() {
+    return shapeValuesource + " " + op + " " + queryShape;
+  }
+
+  @Override
+  public void createWeight(Map context, IndexSearcher searcher) throws IOException {
+    shapeValuesource.createWeight(context, searcher);
+  }
+
+  @Override
+  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
+    final FunctionValues shapeValues = shapeValuesource.getValues(context, readerContext);
+
+    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);
+        List<Explanation> details = new ArrayList<>(Arrays.asList(exp.getDetails()));
+        details.add(shapeValues.explain(doc));
+        return Explanation.match(exp.getValue(), exp.getDescription(), details);
+      }
+    };
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) 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;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = shapeValuesource.hashCode();
+    result = 31 * result + op.hashCode();
+    result = 31 * result + queryShape.hashCode();
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/package-info.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/package-info.java
new file mode 100644
index 0000000..08a872a
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+/**
+ * Advanced spatial utilities.
+ */
+package org.apache.lucene.spatial.util;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java
new file mode 100644
index 0000000..d31fd59
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.vector;
+
+import com.spatial4j.core.distance.DistanceCalculator;
+import com.spatial4j.core.shape.Point;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.util.Bits;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * An implementation of the Lucene ValueSource model that returns the distance
+ * for a {@link PointVectorStrategy}.
+ *
+ * @lucene.internal
+ */
+public class DistanceValueSource extends ValueSource {
+
+  private PointVectorStrategy strategy;
+  private final Point from;
+  private final double multiplier;
+
+  /**
+   * Constructor.
+   */
+  public DistanceValueSource(PointVectorStrategy strategy, Point from, double multiplier) {
+    this.strategy = strategy;
+    this.from = from;
+    this.multiplier = multiplier;
+  }
+
+  /**
+   * Returns the ValueSource description.
+   */
+  @Override
+  public String description() {
+    return "DistanceValueSource("+strategy+", "+from+")";
+  }
+
+  /**
+   * Returns the FunctionValues used by the function query.
+   */
+  @Override
+  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
+    LeafReader reader = readerContext.reader();
+
+    final NumericDocValues ptX = DocValues.getNumeric(reader, strategy.getFieldNameX());
+    final NumericDocValues ptY = DocValues.getNumeric(reader, strategy.getFieldNameY());
+    final Bits validX =  DocValues.getDocsWithField(reader, strategy.getFieldNameX());
+    final Bits validY =  DocValues.getDocsWithField(reader, strategy.getFieldNameY());
+
+    return new FunctionValues() {
+
+      private final Point from = DistanceValueSource.this.from;
+      private final DistanceCalculator calculator = strategy.getSpatialContext().getDistCalc();
+      private final double nullValue =
+          (strategy.getSpatialContext().isGeo() ? 180 * multiplier : Double.MAX_VALUE);
+
+      @Override
+      public float floatVal(int doc) {
+        return (float) doubleVal(doc);
+      }
+
+      @Override
+      public double doubleVal(int doc) {
+        // make sure it has minX and area
+        if (validX.get(doc)) {
+          assert validY.get(doc);
+          return calculator.distance(from, Double.longBitsToDouble(ptX.get(doc)), Double.longBitsToDouble(ptY.get(doc))) * multiplier;
+        }
+        return nullValue;
+      }
+
+      @Override
+      public String toString(int doc) {
+        return description() + "=" + floatVal(doc);
+      }
+    };
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    DistanceValueSource that = (DistanceValueSource) o;
+
+    if (!from.equals(that.from)) return false;
+    if (!strategy.equals(that.strategy)) return false;
+    if (multiplier != that.multiplier) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    return from.hashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
new file mode 100644
index 0000000..f572f82
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.vector;
+
+import com.spatial4j.core.context.SpatialContext;
+import com.spatial4j.core.shape.Circle;
+import com.spatial4j.core.shape.Point;
+import com.spatial4j.core.shape.Rectangle;
+import com.spatial4j.core.shape.Shape;
+import org.apache.lucene.document.LegacyDoubleField;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.queries.function.FunctionRangeQuery;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.LegacyNumericRangeQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.spatial.SpatialStrategy;
+import org.apache.lucene.spatial.query.SpatialArgs;
+import org.apache.lucene.spatial.query.SpatialOperation;
+import org.apache.lucene.spatial.query.UnsupportedSpatialOperation;
+
+/**
+ * Simple {@link SpatialStrategy} which represents Points in two numeric {@link
+ * org.apache.lucene.document.LegacyDoubleField}s.  The Strategy's best feature is decent distance sort.
+ *
+ * <p>
+ * <b>Characteristics:</b>
+ * <br>
+ * <ul>
+ * <li>Only indexes points; just one per field value.</li>
+ * <li>Can query by a rectangle or circle.</li>
+ * <li>{@link
+ * org.apache.lucene.spatial.query.SpatialOperation#Intersects} and {@link
+ * SpatialOperation#IsWithin} is supported.</li>
+ * <li>Uses the FieldCache for
+ * {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point)} and for
+ * searching with a Circle.</li>
+ * </ul>
+ *
+ * <p>
+ * <b>Implementation:</b>
+ * <p>
+ * This is a simple Strategy.  Search works with {@link org.apache.lucene.search.LegacyNumericRangeQuery}s on
+ * an x and y pair of fields.  A Circle query does the same bbox query but adds a
+ * ValueSource filter on
+ * {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point)}.
+ * <p>
+ * 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
+ */
+public class PointVectorStrategy extends SpatialStrategy {
+
+  public static final String SUFFIX_X = "__x";
+  public static final String SUFFIX_Y = "__y";
+
+  private final String fieldNameX;
+  private final String fieldNameY;
+
+  public int precisionStep = 8; // same as solr default
+
+  public PointVectorStrategy(SpatialContext ctx, String fieldNamePrefix) {
+    super(ctx, fieldNamePrefix);
+    this.fieldNameX = fieldNamePrefix+SUFFIX_X;
+    this.fieldNameY = fieldNamePrefix+SUFFIX_Y;
+  }
+
+  public void setPrecisionStep( int p ) {
+    precisionStep = p;
+    if (precisionStep<=0 || precisionStep>=64)
+      precisionStep=Integer.MAX_VALUE;
+  }
+
+  String getFieldNameX() {
+    return fieldNameX;
+  }
+
+  String getFieldNameY() {
+    return fieldNameY;
+  }
+
+  @Override
+  public Field[] createIndexableFields(Shape shape) {
+    if (shape instanceof Point)
+      return createIndexableFields((Point) shape);
+    throw new UnsupportedOperationException("Can only index Point, not " + shape);
+  }
+
+  /** @see #createIndexableFields(com.spatial4j.core.shape.Shape) */
+  public Field[] createIndexableFields(Point point) {
+    FieldType doubleFieldType = new FieldType(LegacyDoubleField.TYPE_NOT_STORED);
+    doubleFieldType.setNumericPrecisionStep(precisionStep);
+    Field[] f = new Field[2];
+    f[0] = new LegacyDoubleField(fieldNameX, point.getX(), doubleFieldType);
+    f[1] = new LegacyDoubleField(fieldNameY, point.getY(), doubleFieldType);
+    return f;
+  }
+
+  @Override
+  public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
+    return new DistanceValueSource(this, queryPoint, multiplier);
+  }
+
+  @Override
+  public ConstantScoreQuery makeQuery(SpatialArgs args) {
+    if(! SpatialOperation.is( args.getOperation(),
+        SpatialOperation.Intersects,
+        SpatialOperation.IsWithin ))
+      throw new UnsupportedSpatialOperation(args.getOperation());
+    Shape shape = args.getShape();
+    if (shape instanceof Rectangle) {
+      Rectangle bbox = (Rectangle) shape;
+      return new ConstantScoreQuery(makeWithin(bbox));
+    } else if (shape instanceof Circle) {
+      Circle circle = (Circle)shape;
+      Rectangle bbox = circle.getBoundingBox();
+      Query approxQuery = makeWithin(bbox);
+      BooleanQuery.Builder bqBuilder = new BooleanQuery.Builder();
+      FunctionRangeQuery vsRangeQuery =
+          new FunctionRangeQuery(makeDistanceValueSource(circle.getCenter()), 0.0, circle.getRadius(), true, true);
+      bqBuilder.add(approxQuery, BooleanClause.Occur.FILTER);//should have lowest "cost" value; will drive iteration
+      bqBuilder.add(vsRangeQuery, BooleanClause.Occur.FILTER);
+      return new ConstantScoreQuery(bqBuilder.build());
+    } else {
+      throw new UnsupportedOperationException("Only Rectangles and Circles are currently supported, " +
+          "found [" + shape.getClass() + "]");//TODO
+    }
+  }
+
+  /**
+   * Constructs a query to retrieve documents that fully contain the input envelope.
+   */
+  private Query makeWithin(Rectangle bbox) {
+    BooleanQuery.Builder bq = new BooleanQuery.Builder();
+    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()), BooleanClause.Occur.SHOULD );
+      bq.add(rangeQuery(fieldNameX, bbox.getMinX(), null/*+180*/), BooleanClause.Occur.SHOULD );
+      bq.setMinimumNumberShouldMatch(1);//must match at least one of the SHOULD
+    } else {
+      bq.add(rangeQuery(fieldNameX, bbox.getMinX(), bbox.getMaxX()), MUST);
+    }
+    bq.add(rangeQuery(fieldNameY, bbox.getMinY(), bbox.getMaxY()), MUST);
+    return bq.build();
+  }
+
+  private LegacyNumericRangeQuery<Double> rangeQuery(String fieldName, Double min, Double max) {
+    return LegacyNumericRangeQuery.newDoubleRange(
+        fieldName,
+        precisionStep,
+        min,
+        max,
+        true,
+        true);//inclusive
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/package-info.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/package-info.java
new file mode 100644
index 0000000..f8dffe2
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+/**
+ * Spatial strategy that uses two fields.
+ */
+package org.apache.lucene.spatial.vector;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/overview.html
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/overview.html b/lucene/spatial-extras/src/java/overview.html
new file mode 100644
index 0000000..1b7afd1
--- /dev/null
+++ b/lucene/spatial-extras/src/java/overview.html
@@ -0,0 +1,67 @@
+<!--
+ 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.
+-->
+<html>
+  <head>
+    <title>Apache Lucene Spatial-Extras Module</title>
+  </head>
+  <body>
+
+  <h1>The Spatial-Extras Module for Apache Lucene</h1>
+
+  <p>
+    The spatial-extras module, new to Lucene 6.5, is the new home for the original
+    lucene <a href="../spatial/overview-summary.html">spatial</a> module.
+    The principle interface to this module is a {@link org.apache.lucene.spatial.SpatialStrategy}
+    which encapsulates an approach to indexing and searching
+    based on shapes.  Different Strategies have different features and
+    performance profiles, which are documented at each Strategy implementation
+    class level.
+  </p>
+  <p>
+    For some sample code showing how to use the API, see
+      SpatialExample.java in the tests.
+  </p>
+<p>
+    The spatial-extras module uses
+    <a href="https://github.com/spatial4j/spatial4j">Spatial4j</a>
+    heavily.  Spatial4j is an ASL licensed library with these capabilities:
+    <ul>
+    <li>Provides shape implementations, namely point, rectangle,
+      and circle.  Both geospatial contexts and plain 2D Euclidean/Cartesian contexts
+      are supported.
+      With an additional dependency, it adds polygon and other geometry shape
+      support via integration with
+      <a href="http://sourceforge.net/projects/jts-topo-suite/">JTS Topology Suite</a>.
+      This includes dateline wrap support.</li>
+    <li>Shape parsing and serialization, including
+      <a href="http://en.wikipedia.org/wiki/Well-known_text">Well-Known Text (WKT)</a>
+      (via JTS).</li>
+    <li>Distance and other spatial related math calculations.</li>
+    </ul>
+  </p>
+  <p>
+    Historical note: The new spatial-extras module was once known as
+    Lucene Spatial Playground (LSP) as an external project.  In ~March 2012, LSP
+    split into the spatial module as part of Lucene and Spatial4j externally. A
+    large chunk of the LSP implementation originated as SOLR-2155 which uses
+    trie/prefix-tree algorithms with a geohash encoding.  That approach is
+    implemented in {@link org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy}
+    today.
+  </p>
+
+  </body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/test-files/cities-Intersects-BBox.txt
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test-files/cities-Intersects-BBox.txt b/lucene/spatial-extras/src/test-files/cities-Intersects-BBox.txt
new file mode 100644
index 0000000..e85748c
--- /dev/null
+++ b/lucene/spatial-extras/src/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/lucene-solr/blob/89db4950/lucene/spatial-extras/src/test-files/data/LUCENE-4464.txt
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test-files/data/LUCENE-4464.txt b/lucene/spatial-extras/src/test-files/data/LUCENE-4464.txt
new file mode 100644
index 0000000..dfb5a40
--- /dev/null
+++ b/lucene/spatial-extras/src/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