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

svn commit: r1291499 [3/12] - in /lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene: ./ src/ src/main/ src/main/java/ src/main/java/org/ src/main/java/org/apache/ src/main/java/org/apache/lucene/ src/main/java/org/apache/lucene/s...

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/prefix/quad/QuadPrefixTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/prefix/quad/QuadPrefixTree.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/prefix/quad/QuadPrefixTree.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/prefix/quad/QuadPrefixTree.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,301 @@
+/*
+ * 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.base.prefix.quad;
+
+import org.apache.lucene.spatial.base.prefix.SpatialPrefixTreeFactory;
+import org.apache.lucene.spatial.base.shape.SpatialRelation;
+import org.apache.lucene.spatial.base.context.SpatialContext;
+import org.apache.lucene.spatial.base.prefix.Node;
+import org.apache.lucene.spatial.base.prefix.SpatialPrefixTree;
+import org.apache.lucene.spatial.base.shape.Point;
+import org.apache.lucene.spatial.base.shape.Rectangle;
+import org.apache.lucene.spatial.base.shape.Shape;
+import org.apache.lucene.spatial.base.shape.simple.PointImpl;
+
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+public class QuadPrefixTree extends SpatialPrefixTree {
+
+  public static class Factory extends SpatialPrefixTreeFactory {
+
+    @Override
+    protected int getLevelForDistance(double degrees) {
+      QuadPrefixTree grid = new QuadPrefixTree(ctx, MAX_LEVELS_POSSIBLE);
+      return grid.getLevelForDistance(degrees) + 1;//returns 1 greater
+    }
+
+    @Override
+    protected SpatialPrefixTree newSPT() {
+      return new QuadPrefixTree(ctx,
+          maxLevels != null ? maxLevels : MAX_LEVELS_POSSIBLE);
+    }
+  }
+
+  public static final int MAX_LEVELS_POSSIBLE = 50;//not really sure how big this should be
+
+  public static final int DEFAULT_MAX_LEVELS = 12;
+  private final double xmin;
+  private final double xmax;
+  private final double ymin;
+  private final double ymax;
+  private final double xmid;
+  private final double ymid;
+
+  private final double gridW;
+  public final double gridH;
+
+  final double[] levelW;
+  final double[] levelH;
+  final int[]    levelS; // side
+  final int[]    levelN; // number
+
+  public QuadPrefixTree(
+      SpatialContext ctx, Rectangle bounds, int maxLevels) {
+    super(ctx, maxLevels);
+    this.xmin = bounds.getMinX();
+    this.xmax = bounds.getMaxX();
+    this.ymin = bounds.getMinY();
+    this.ymax = bounds.getMaxY();
+
+    levelW = new double[maxLevels];
+    levelH = new double[maxLevels];
+    levelS = new int[maxLevels];
+    levelN = new int[maxLevels];
+
+    gridW = xmax - xmin;
+    gridH = ymax - ymin;
+    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;
+      levelH[i] = levelH[i - 1] / 2.0;
+      levelS[i] = levelS[i - 1] * 2;
+      levelN[i] = levelN[i - 1] * 4;
+    }
+  }
+
+  public QuadPrefixTree(SpatialContext ctx) {
+    this(ctx, DEFAULT_MAX_LEVELS);
+  }
+
+  public QuadPrefixTree(
+      SpatialContext ctx, int maxLevels) {
+    this(ctx, ctx.getWorldBounds(), maxLevels);
+  }
+
+  public void printInfo() {
+    NumberFormat nf = NumberFormat.getNumberInstance();
+    nf.setMaximumFractionDigits(5);
+    nf.setMinimumFractionDigits(5);
+    nf.setMinimumIntegerDigits(3);
+
+    for (int i = 0; i < maxLevels; i++) {
+      System.out.println(i + "]\t" + nf.format(levelW[i]) + "\t" + nf.format(levelH[i]) + "\t" +
+          levelS[i] + "\t" + (levelS[i] * levelS[i]));
+    }
+  }
+
+  @Override
+  public int getLevelForDistance(double dist) {
+    for (int i = 1; i < maxLevels; i++) {
+      //note: level[i] is actually a lookup for level i+1
+      if(dist > levelW[i] || dist > levelH[i]) {
+        return i;
+      }
+    }
+    return maxLevels;
+  }
+
+  @Override
+  public Node getNode(Point p, int level) {
+    List<Node> cells = new ArrayList<Node>(1);
+    build(xmid, ymid, 0, cells, new StringBuilder(), new PointImpl(p.getX(),p.getY()), level);
+    return cells.get(0);//note cells could be longer if p on edge
+  }
+
+  @Override
+  public Node getNode(String token) {
+    return new QuadCell(token);
+  }
+
+  @Override
+  public Node getNode(byte[] bytes, int offset, int len) {
+    return new QuadCell(bytes, offset, len);
+  }
+
+  @Override //for performance
+  public List<Node> getNodes(Shape shape, int detailLevel, boolean inclParents) {
+    if (shape instanceof Point)
+      return super.getNodesAltPoint((Point) shape, detailLevel, inclParents);
+    else
+      return super.getNodes(shape, detailLevel, inclParents);
+  }
+
+  private void build(
+      double x,
+      double y,
+      int level,
+      List<Node> matches,
+      StringBuilder str,
+      Shape shape,
+      int maxLevel) {
+    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);
+    checkBattenberg('B', x + w, y + h, level, matches, str, shape, maxLevel);
+    checkBattenberg('C', x - w, y - h, level, matches, str, shape, maxLevel);
+    checkBattenberg('D', x + w, y - h, level, matches, str, shape, maxLevel);
+
+    // possibly consider hilbert curve
+    // 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,
+      List<Node> matches,
+      StringBuilder str,
+      Shape shape,
+      int maxLevel) {
+    assert str.length() == level;
+    double w = levelW[level] / 2;
+    double h = levelH[level] / 2;
+
+    int strlen = str.length();
+    Rectangle rectangle = ctx.makeRect(cx - w, cx + w, cy - h, cy + h);
+    SpatialRelation v = shape.relate(rectangle, ctx);
+    if (SpatialRelation.CONTAINS == v) {
+      str.append(c);
+      //str.append(SpatialPrefixGrid.COVER);
+      matches.add(new QuadCell(str.toString(),v.transpose()));
+    } else 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(str.toString(),v.transpose()));
+      } else {
+        build(cx, cy, nextLevel, matches, str, shape, maxLevel);
+      }
+    }
+    str.setLength(strlen);
+  }
+
+  class QuadCell extends Node {
+
+    public QuadCell(String token) {
+      super(QuadPrefixTree.this, token);
+    }
+
+    public QuadCell(String token, SpatialRelation shapeRel) {
+      super(QuadPrefixTree.this, token);
+      this.shapeRel = shapeRel;
+    }
+
+    QuadCell(byte[] bytes, int off, int len) {
+      super(QuadPrefixTree.this, bytes, off, len);
+    }
+
+    @Override
+    public void reset(byte[] bytes, int off, int len) {
+      super.reset(bytes, off, len);
+      shape = null;
+    }
+
+    @Override
+    public Collection<Node> getSubCells() {
+      List<Node> cells = new ArrayList<Node>(4);
+      cells.add(new QuadCell(getTokenString()+"A"));
+      cells.add(new QuadCell(getTokenString()+"B"));
+      cells.add(new QuadCell(getTokenString()+"C"));
+      cells.add(new QuadCell(getTokenString()+"D"));
+      return cells;
+    }
+
+    @Override
+    public int getSubCellsSize() {
+      return 4;
+    }
+
+    @Override
+    public Node getSubCell(Point p) {
+      return QuadPrefixTree.this.getNode(p,getLevel()+1);//not performant!
+    }
+
+    private Shape shape;//cache
+
+    @Override
+    public Shape getShape() {
+      if (shape == null)
+        shape = makeShape();
+      return shape;
+    }
+
+    private Rectangle makeShape() {
+      String token = getTokenString();
+      double xmin = QuadPrefixTree.this.xmin;
+      double ymin = QuadPrefixTree.this.ymin;
+
+      for (int i = 0; i < token.length(); i++) {
+        char c = token.charAt(i);
+        if ('A' == c || 'a' == c) {
+          ymin += levelH[i];
+        } else if ('B' == c || 'b' == c) {
+          xmin += levelW[i];
+          ymin += levelH[i];
+        } else if ('C' == c || 'c' == c) {
+          // nothing really
+        }
+        else if('D' == c || 'd' == c) {
+          xmin += levelW[i];
+        } else {
+          throw new RuntimeException("unexpected char: " + c);
+        }
+      }
+      int len = token.length();
+      double width, height;
+      if (len > 0) {
+        width = levelW[len-1];
+        height = levelH[len-1];
+      } else {
+        width = gridW;
+        height = gridH;
+      }
+      return ctx.makeRect(xmin, xmin + width, ymin, ymin + height);
+    }
+  }//QuadCell
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/query/SpatialArgs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/query/SpatialArgs.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/query/SpatialArgs.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/query/SpatialArgs.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,134 @@
+/*
+ * 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.base.query;
+
+import org.apache.lucene.spatial.base.context.SpatialContext;
+import org.apache.lucene.spatial.base.context.simple.SimpleSpatialContext;
+import org.apache.lucene.spatial.base.distance.DistanceUnits;
+import org.apache.lucene.spatial.base.exception.InvalidSpatialArgument;
+import org.apache.lucene.spatial.base.shape.Shape;
+
+public class SpatialArgs {
+
+  public static final double DEFAULT_DIST_PRECISION = 0.025d;
+
+  private SpatialOperation operation;
+  private Shape shape;
+  private double distPrecision = DEFAULT_DIST_PRECISION;
+
+  // Useful for 'distance' calculations
+  private Double min;
+  private Double max;
+
+  public SpatialArgs(SpatialOperation operation) {
+    this.operation = operation;
+  }
+
+  public SpatialArgs(SpatialOperation operation, Shape shape) {
+    this.operation = operation;
+    this.shape = shape;
+  }
+
+  /**
+   * Check if the arguments make sense -- throw an exception if not
+   */
+  public void validate() throws InvalidSpatialArgument {
+    if (operation.isTargetNeedsArea() && !shape.hasArea()) {
+      throw new InvalidSpatialArgument(operation + " only supports geometry with area");
+    }
+  }
+
+  public String toString( SpatialContext context ) {
+    StringBuilder str = new StringBuilder();
+    str.append( operation.getName() ).append( '(' );
+    str.append( context.toString( shape ) );
+    if( min != null ) {
+      str.append(" min=").append(min);
+    }
+    if( max != null ) {
+      str.append(" max=").append(max);
+    }
+    str.append(" distPrec=").append(String.format("%.2f%%", distPrecision/100d));
+    str.append( ')' );
+    return str.toString();
+  }
+
+  @Override
+  public String toString()
+  {
+    return toString( new SimpleSpatialContext(DistanceUnits.KILOMETERS) );
+  }
+
+  //------------------------------------------------
+  // Getters & Setters
+  //------------------------------------------------
+
+  public SpatialOperation getOperation() {
+    return operation;
+  }
+
+  public void setOperation(SpatialOperation operation) {
+    this.operation = operation;
+  }
+
+  /**
+   * Considers {@link SpatialOperation#BBoxWithin} in returning the shape.
+   * @return
+   */
+  public Shape getShape() {
+    if (shape != null && (operation == SpatialOperation.BBoxWithin || operation == SpatialOperation.BBoxIntersects))
+      return shape.getBoundingBox();
+    return shape;
+  }
+
+  public void setShape(Shape shape) {
+    this.shape = shape;
+  }
+
+  /**
+   * The fraction of the distance from the center of the query shape to its nearest edge that is considered acceptable
+   * error. The algorithm for computing the distance to the nearest edge is actually a little different. It normalizes
+   * the shape to a square given it's bounding box area:
+   * <pre>sqrt(shape.bbox.area)/2</pre>
+   * And the error distance is beyond the shape such that the shape is a minimum shape.
+   */
+  public Double getDistPrecision() {
+    return distPrecision;
+  }
+
+  public void setDistPrecision(Double distPrecision) {
+    if (distPrecision != null)
+      this.distPrecision = distPrecision;
+  }
+
+  public Double getMin() {
+    return min;
+  }
+
+  public void setMin(Double min) {
+    this.min = min;
+  }
+
+  public Double getMax() {
+    return max;
+  }
+
+  public void setMax(Double max) {
+    this.max = max;
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/query/SpatialArgsParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/query/SpatialArgsParser.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/query/SpatialArgsParser.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/query/SpatialArgsParser.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,88 @@
+/*
+ * 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.base.query;
+
+import org.apache.lucene.spatial.base.context.SpatialContext;
+import org.apache.lucene.spatial.base.exception.InvalidShapeException;
+import org.apache.lucene.spatial.base.exception.InvalidSpatialArgument;
+import org.apache.lucene.spatial.base.shape.Shape;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.StringTokenizer;
+
+public class SpatialArgsParser
+{
+  public SpatialArgs parse(String v, SpatialContext ctx) throws InvalidSpatialArgument, InvalidShapeException {
+    int idx = v.indexOf('(');
+    int edx = v.lastIndexOf(')');
+
+    if (idx < 0 || idx > edx) {
+      throw new InvalidSpatialArgument("missing parens: " + v, null);
+    }
+
+    SpatialOperation op = SpatialOperation.get(v.substring(0, idx).trim());
+
+    String body = v.substring(idx + 1, edx).trim();
+    if (body.length() < 1) {
+      throw new InvalidSpatialArgument("missing body : " + v, null);
+    }
+
+    Shape shape = ctx.readShape(body);
+    SpatialArgs args = new SpatialArgs(op,shape);
+
+    if (v.length() > (edx + 1)) {
+      body = v.substring( edx+1 ).trim();
+      if (body.length() > 0) {
+        Map<String,String> aa = parseMap(body);
+        args.setMin(readDouble(aa.remove("min")) );
+        args.setMax(readDouble(aa.remove("max")));
+        args.setDistPrecision(readDouble(aa.remove("distPrec")));
+        if (!aa.isEmpty()) {
+          throw new InvalidSpatialArgument("unused parameters: " + aa, null);
+        }
+      }
+    }
+    return args;
+  }
+
+  protected static Double readDouble(String v) {
+      return v == null ? null : Double.valueOf(v);
+  }
+
+  protected static boolean readBool(String v, boolean defaultValue) {
+      return v == null ? defaultValue : Boolean.parseBoolean(v);
+  }
+
+  protected static Map<String,String> parseMap(String body) {
+    Map<String,String> map = new HashMap<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.put(k, v);
+      } else {
+        map.put(a, a);
+      }
+    }
+    return map;
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/query/SpatialOperation.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/query/SpatialOperation.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/query/SpatialOperation.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/query/SpatialOperation.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,107 @@
+/* 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.
+ */
+package org.apache.lucene.spatial.base.query;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+import org.apache.lucene.spatial.base.exception.InvalidSpatialArgument;
+
+/**
+ * A clause that compares a stored geometry to a supplied geometry.
+ */
+public class SpatialOperation implements Serializable {
+  // Private registry
+  private static final Map<String, SpatialOperation> registry = new HashMap<String, SpatialOperation>();
+  private static final List<SpatialOperation> list = new ArrayList<SpatialOperation>();
+
+  // Geometry Operations
+  public static final SpatialOperation BBoxIntersects = new SpatialOperation("BBoxIntersects", true, false, false);
+  public static final SpatialOperation BBoxWithin     = new SpatialOperation("BBoxWithin", true, false, false);
+  public static final SpatialOperation Contains       = new SpatialOperation("Contains", true, true, false);
+  public static final SpatialOperation Intersects     = new SpatialOperation("Intersects", true, false, false);
+  public static final SpatialOperation IsEqualTo      = new SpatialOperation("IsEqualTo", false, false, false);
+  public static final SpatialOperation IsDisjointTo   = new SpatialOperation("IsDisjointTo", false, false, false);
+  public static final SpatialOperation IsWithin       = new SpatialOperation("IsWithin", true, false, true);
+  public static final SpatialOperation Overlaps       = new SpatialOperation("Overlaps", true, false, true);
+
+  // Member variables
+  private final boolean scoreIsMeaningful;
+  private final boolean sourceNeedsArea;
+  private final boolean targetNeedsArea;
+  private final String name;
+
+  protected SpatialOperation(String name, boolean scoreIsMeaningful, boolean sourceNeedsArea, boolean targetNeedsArea) {
+    this.name = name;
+    this.scoreIsMeaningful = scoreIsMeaningful;
+    this.sourceNeedsArea = sourceNeedsArea;
+    this.targetNeedsArea = targetNeedsArea;
+    registry.put(name, this);
+    registry.put(name.toUpperCase(Locale.US), this);
+    list.add( this );
+  }
+
+  public static SpatialOperation get( String v ) {
+    SpatialOperation op = registry.get( v );
+    if( op == null ) {
+      op = registry.get(v.toUpperCase(Locale.US));
+    }
+    if( op == null ) {
+      throw new InvalidSpatialArgument("Unknown Operation: " + v );
+    }
+    return op;
+  }
+
+  public static List<SpatialOperation> values() {
+    return list;
+  }
+
+  public static boolean is( SpatialOperation op, SpatialOperation ... tst ) {
+    for( SpatialOperation t : tst ) {
+      if( op == t ) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+
+  // ================================================= Getters / Setters =============================================
+
+  public boolean isScoreIsMeaningful() {
+    return scoreIsMeaningful;
+  }
+
+  public boolean isSourceNeedsArea() {
+    return sourceNeedsArea;
+  }
+
+  public boolean isTargetNeedsArea() {
+    return targetNeedsArea;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public String toString() {
+    return name;
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/query/package-info.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/query/package-info.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/query/package-info.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/query/package-info.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+
+/**
+ * Lucene spatial Query options
+ * <ul>
+ *  <li>useful for client side requets</li>
+ * </ul>
+ */
+package org.apache.lucene.spatial.base.query;
+

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/Circle.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/Circle.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/Circle.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/Circle.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,25 @@
+/*
+ * 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.base.shape;
+
+/**
+ * This is basically a circle.
+ */
+public interface Circle extends Shape {
+  double getDistance();
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/MultiShape.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/MultiShape.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/MultiShape.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/MultiShape.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,108 @@
+/*
+ * 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.base.shape;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.lucene.spatial.base.context.SpatialContext;
+
+import java.util.Collection;
+
+/**
+ * A collection of Shape objects.
+ */
+public class MultiShape implements Shape {
+  private final Collection<Shape> geoms;
+  private final Rectangle bbox;
+
+  public MultiShape(Collection<Shape> geoms, SpatialContext ctx) {
+    this.geoms = geoms;
+    double minX = Double.MAX_VALUE;
+    double minY = Double.MAX_VALUE;
+    double maxX = Double.MIN_VALUE;
+    double maxY = Double.MIN_VALUE;
+    for (Shape geom : geoms) {
+      Rectangle r = geom.getBoundingBox();
+      minX = Math.min(minX,r.getMinX());
+      minY = Math.min(minY,r.getMinY());
+      maxX = Math.max(maxX,r.getMaxX());
+      maxY = Math.max(maxY,r.getMaxY());
+    }
+    this.bbox = ctx.makeRect(minX, maxX, minY, maxY);
+  }
+
+  @Override
+  public Rectangle getBoundingBox() {
+    return bbox;
+  }
+
+  @Override
+  public Point getCenter() {
+    return bbox.getCenter();
+  }
+
+  @Override
+  public boolean hasArea() {
+    for (Shape geom : geoms) {
+      if( geom.hasArea() ) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public SpatialRelation relate(Shape other, SpatialContext ctx) {
+    boolean allOutside = true;
+    boolean allContains = true;
+    for (Shape geom : geoms) {
+      SpatialRelation sect = geom.relate(other, ctx);
+      if (sect != SpatialRelation.DISJOINT)
+        allOutside = false;
+      if (sect != SpatialRelation.CONTAINS)
+        allContains = false;
+      if (!allContains && !allOutside)
+        return SpatialRelation.INTERSECTS;//short circuit
+    }
+    if (allOutside)
+      return SpatialRelation.DISJOINT;
+    if (allContains)
+      return SpatialRelation.CONTAINS;
+    return SpatialRelation.INTERSECTS;
+  }
+
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null) { return false; }
+    if (obj == this) { return true; }
+    if (obj.getClass() != getClass()) {
+      return false;
+    }
+    MultiShape rhs = (MultiShape) obj;
+    return new EqualsBuilder()
+                  .append(geoms, rhs.geoms)
+                  .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(83, 29).append(geoms.hashCode()).
+      toHashCode();
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/Point.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/Point.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/Point.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/Point.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,24 @@
+/*
+ * 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.base.shape;
+
+public interface Point extends Shape {
+
+  public double getX();
+  public double getY();
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/Rectangle.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/Rectangle.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/Rectangle.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/Rectangle.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,40 @@
+/*
+ * 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.base.shape;
+
+import org.apache.lucene.spatial.base.context.SpatialContext;
+
+public interface Rectangle extends Shape {
+
+  public double getWidth();
+  public double getHeight();
+
+  public double getMinX();
+  public double getMinY();
+  public double getMaxX();
+  public double getMaxY();
+
+  /** If {@link #hasArea()} then this returns the area, otherwise it returns 0. */
+  public double getArea();
+  /** Only meaningful for geospatial contexts. */
+  public boolean getCrossesDateLine();
+
+  /* There is no axis line shape, and this is more efficient then creating a flat Rectangle for intersect(). */
+  public SpatialRelation relate_yRange(double minY, double maxY, SpatialContext ctx);
+  public SpatialRelation relate_xRange(double minX, double maxX, SpatialContext ctx);
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/Shape.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/Shape.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/Shape.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/Shape.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,48 @@
+/*
+ * 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.base.shape;
+
+import org.apache.lucene.spatial.base.context.SpatialContext;
+
+public interface Shape {
+
+  /**
+   * Describe the relationship between the two objects.  For example
+   *
+   *   this is WITHIN other
+   *   this CONTAINS other
+   *   this is DISJOINT other
+   *   this INTERSECTS other
+   *
+   * The context object is optional -- it may include spatial reference.
+   */
+  SpatialRelation relate(Shape other, SpatialContext ctx);
+
+  /**
+   * Get the bounding box for this Shape
+   */
+  Rectangle getBoundingBox();
+
+  /**
+   * @return true if the shape has area.  This will be false for points and lines
+   */
+  boolean hasArea();
+
+  Point getCenter();
+}
+

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/SpatialRelation.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/SpatialRelation.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/SpatialRelation.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/SpatialRelation.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,69 @@
+/*
+ * 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.base.shape;
+
+/**
+ * The set of spatial relationships.  Naming is consistent with OGC spec conventions as seen in SQL/MM and others.
+ * No equality case.  If two Shape instances are equal then the result might be CONTAINS or WITHIN, and
+ * some logic might fail under this edge condition when it's not careful to check.
+ * Client code must be written to detect this and act accordingly.  In RectangleImpl.relate(), it checks
+ * for this explicitly, for example.  TestShapes2D.assertRelation() checks too.
+ */
+public enum SpatialRelation {
+  WITHIN,
+  CONTAINS,
+  DISJOINT,
+  INTERSECTS;
+  //Don't have these: TOUCHES, CROSSES, OVERLAPS
+
+  public SpatialRelation transpose() {
+    switch(this) {
+      case CONTAINS: return SpatialRelation.WITHIN;
+      case WITHIN: return SpatialRelation.CONTAINS;
+      default: return this;
+    }
+  }
+
+  /**
+   * If you were to call aShape.relate(bShape) and aShape.relate(cShape), you could call
+   * this to merge the intersect results as if bShape & cShape were combined into {@link MultiShape}.
+   * @param other
+   * @return
+   */
+  public SpatialRelation combine(SpatialRelation other) {
+    if (this == other)
+      return this;
+    if (this == WITHIN || other == WITHIN)
+      return WITHIN;
+    return INTERSECTS;
+  }
+
+  public boolean intersects() {
+    return this != DISJOINT;
+  }
+
+  /** Not commutative!  WITHIN.inverse().inverse() != WITHIN. */
+  public SpatialRelation inverse() {
+    switch(this) {
+      case DISJOINT: return CONTAINS;
+      case CONTAINS: return DISJOINT;
+      case WITHIN: return INTERSECTS;//not commutative!
+    }
+    return INTERSECTS;
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/simple/CircleImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/simple/CircleImpl.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/simple/CircleImpl.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/simple/CircleImpl.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,234 @@
+/*
+ * 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.base.shape.simple;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.lucene.spatial.base.context.SpatialContext;
+import org.apache.lucene.spatial.base.shape.*;
+
+/**
+ * A circle, also known as a point-radius, based on a
+ * {@link org.apache.lucene.spatial.base.distance.DistanceCalculator} which does all the work. This implementation
+ * should work for both cartesian 2D and geodetic sphere surfaces.
+ * Threadsafe & immutable.
+ */
+public class CircleImpl implements Circle {
+
+  protected final Point point;
+  protected final double distance;
+
+  protected final SpatialContext ctx;
+
+  /* below is calculated & cached: */
+  
+  protected final Rectangle enclosingBox;
+
+  //we don't have a line shape so we use a rectangle for these axis
+
+  public CircleImpl(Point p, double dist, SpatialContext ctx) {
+    //We assume any normalization / validation of params already occurred (including bounding dist)
+    this.point = p;
+    this.distance = dist;
+    this.ctx = ctx;
+    this.enclosingBox = ctx.getDistCalc().calcBoxByDistFromPt(point, distance, ctx);
+  }
+
+  public Point getCenter() {
+    return point;
+  }
+
+  @Override
+  public double getDistance() {
+    return distance;
+  }
+
+  public boolean contains(double x, double y) {
+    return ctx.getDistCalc().distance(point, x, y) <= distance;
+  }
+
+  @Override
+  public boolean hasArea() {
+    return distance > 0;
+  }
+
+  /**
+   * Note that the bounding box might contain a minX that is > maxX, due to WGS84 dateline.
+   * @return
+   */
+  @Override
+  public Rectangle getBoundingBox() {
+    return enclosingBox;
+  }
+
+  @Override
+  public SpatialRelation relate(Shape other, SpatialContext ctx) {
+    assert this.ctx == ctx;
+//This shortcut was problematic in testing due to distinctions of CONTAINS/WITHIN for no-area shapes (lines, points).
+//    if (distance == 0) {
+//      return point.relate(other,ctx).intersects() ? SpatialRelation.WITHIN : SpatialRelation.DISJOINT;
+//    }
+
+    if (other instanceof Point) {
+      return relate((Point) other, ctx);
+    }
+    if (other instanceof Rectangle) {
+      return relate((Rectangle) other, ctx);
+    }
+    if (other instanceof Circle) {
+      return relate((Circle) other, ctx);
+    }
+    return other.relate(this, ctx).transpose();
+  }
+
+  public SpatialRelation relate(Point point, SpatialContext ctx) {
+    return contains(point.getX(),point.getY()) ? SpatialRelation.CONTAINS : SpatialRelation.DISJOINT;
+  }
+
+  public SpatialRelation relate(Rectangle r, SpatialContext ctx) {
+    //Note: Surprisingly complicated!
+
+    //--We start by leveraging the fact we have a calculated bbox that is "cheaper" than use of DistanceCalculator.
+    final SpatialRelation bboxSect = enclosingBox.relate(r, ctx);
+    if (bboxSect == SpatialRelation.DISJOINT || bboxSect == SpatialRelation.WITHIN)
+      return bboxSect;
+    else if (bboxSect == SpatialRelation.CONTAINS && enclosingBox.equals(r))//nasty identity edge-case
+      return SpatialRelation.WITHIN;
+    //bboxSect is INTERSECTS or CONTAINS
+    //The result can be DISJOINT, CONTAINS, or INTERSECTS (not WITHIN)
+
+    return relateRectanglePhase2(r, bboxSect, ctx);
+  }
+
+  protected SpatialRelation relateRectanglePhase2(final Rectangle r, SpatialRelation bboxSect, SpatialContext ctx) {
+    /*
+     !! DOES NOT WORK WITH GEO CROSSING DATELINE OR WORLD-WRAP.
+     TODO upgrade to handle crossing dateline, but not world-wrap; use some x-shifting code from RectangleImpl.
+     */
+
+    //At this point, the only thing we are certain of is that circle is *NOT* WITHIN r, since the bounding box of a
+    // circle MUST be within r for the circle to be within r.
+
+    //--Quickly determine if they are DISJOINT or not.
+    //see http://stackoverflow.com/questions/401847/circle-rectangle-collision-detection-intersection/1879223#1879223
+    final double closestX;
+    double ctr_x = getXAxis();
+    if ( ctr_x < r.getMinX() )
+      closestX = r.getMinX();
+    else if (ctr_x > r.getMaxX())
+      closestX = r.getMaxX();
+    else
+      closestX = ctr_x;
+
+    final double closestY;
+    double ctr_y = getYAxis();
+    if ( ctr_y < r.getMinY() )
+      closestY = r.getMinY();
+    else if (ctr_y > r.getMaxY())
+      closestY = r.getMaxY();
+    else
+      closestY = ctr_y;
+
+    //Check if there is an intersection from this circle to closestXY
+    boolean didContainOnClosestXY = false;
+    if (ctr_x == closestX) {
+      double deltaY = Math.abs(ctr_y - closestY);
+      double distYCirc = (ctr_y < closestY ? enclosingBox.getMaxY() - ctr_y : ctr_y - enclosingBox.getMinY());
+      if (deltaY > distYCirc)
+        return SpatialRelation.DISJOINT;
+    } else if (ctr_y == closestY) {
+      double deltaX = Math.abs(ctr_x - closestX);
+      double distXCirc = (ctr_x < closestX ? enclosingBox.getMaxX() - ctr_x : ctr_x - enclosingBox.getMinX());
+      if (deltaX > distXCirc)
+        return SpatialRelation.DISJOINT;
+    } else {
+      //fallback on more expensive calculation
+      didContainOnClosestXY = true;
+      if(! contains(closestX,closestY) )
+        return SpatialRelation.DISJOINT;
+    }
+
+    //At this point we know that it's *NOT* DISJOINT, so there is some level of intersection. It's *NOT* WITHIN either.
+    // The only question left is whether circle CONTAINS r or simply intersects it.
+
+    //If circle contains r, then its bbox MUST also CONTAIN r.
+    if (bboxSect != SpatialRelation.CONTAINS)
+      return SpatialRelation.INTERSECTS;
+
+    //Find the farthest point of r away from the center of the circle. If that point is contained, then all of r is
+    // contained.
+    double farthestX = r.getMaxX() - ctr_x > ctr_x - r.getMinX() ? r.getMaxX() : r.getMinX();
+    double farthestY = r.getMaxY() - ctr_y > ctr_y - r.getMinY() ? r.getMaxY() : r.getMinY();
+    if (contains(farthestX,farthestY))
+      return SpatialRelation.CONTAINS;
+    return SpatialRelation.INTERSECTS;
+  }
+
+  /**
+   * The y axis horizontal of maximal left-right extent of the circle.
+   */
+  protected double getYAxis() {
+    return point.getY();
+  }
+
+  protected double getXAxis() {
+    return point.getX();
+  }
+
+  public SpatialRelation relate(Circle circle, SpatialContext ctx) {
+    double crossDist = ctx.getDistCalc().distance(point, circle.getCenter());
+    double aDist = distance, bDist = circle.getDistance();
+    if (crossDist > aDist + bDist)
+      return SpatialRelation.DISJOINT;
+    if (crossDist < aDist && crossDist + bDist <= aDist)
+      return SpatialRelation.CONTAINS;
+    if (crossDist < bDist && crossDist + aDist <= bDist)
+      return SpatialRelation.WITHIN;
+
+    return SpatialRelation.INTERSECTS;
+  }
+
+  @Override
+  public String toString() {
+    return "Circle(" + point + ",d=" + distance + ')';
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null) { return false; }
+    if (obj == this) { return true; }
+    if (obj.getClass() != getClass()) {
+      return false;
+    }
+    CircleImpl rhs = (CircleImpl) obj;
+    return new EqualsBuilder()
+                  .append(point, rhs.point)
+                  .append(distance, rhs.distance)
+                  .append(ctx, rhs.ctx)
+                  .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(11, 97).
+      append(point).
+      append(distance).
+      append(ctx).
+      toHashCode();
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/simple/GeoCircleImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/simple/GeoCircleImpl.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/simple/GeoCircleImpl.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/simple/GeoCircleImpl.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,226 @@
+/*
+ * 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.base.shape.simple;
+
+import org.apache.lucene.spatial.base.context.SpatialContext;
+import org.apache.lucene.spatial.base.shape.SpatialRelation;
+import org.apache.lucene.spatial.base.shape.Point;
+import org.apache.lucene.spatial.base.shape.Rectangle;
+
+/**
+ * @author David Smiley - dsmiley@mitre.org
+ */
+public class GeoCircleImpl extends CircleImpl {
+  private final double distDEG;// [0 TO 180]
+  private final GeoCircleImpl inverseCircle;//when distance reaches > 1/2 way around the world, cache the inverse.
+  private final double horizAxisY;//see getYAxis
+
+  public GeoCircleImpl(Point p, double dist, SpatialContext ctx) {
+    super(p, dist, ctx);
+    assert ctx.isGeo();
+
+    //In the direction of latitude (N,S), distance is the same number of degrees.
+    distDEG = ctx.getDistCalc().distanceToDegrees(distance);
+
+    if (distDEG > 90) {
+      assert enclosingBox.getWidth() == 360;
+      double backDistDEG = 180 - distDEG;
+      if (backDistDEG >= 0) {
+        double backDistance = ctx.getDistCalc().degreesToDistance(backDistDEG);
+        Point backPoint = ctx.makePoint(getCenter().getX() + 180, getCenter().getY() + 180);
+        inverseCircle = new GeoCircleImpl(backPoint,backDistance,ctx);
+      } else
+        inverseCircle = null;//whole globe
+      horizAxisY = getCenter().getY();//although probably not used
+    } else {
+      inverseCircle = null;
+      double _horizAxisY = ctx.getDistCalc().calcBoxByDistFromPtHorizAxis(getCenter(), distance, ctx);
+      //some rare numeric conditioning cases can cause this to be barely beyond the box
+      if (_horizAxisY > enclosingBox.getMaxY()) {
+        horizAxisY = enclosingBox.getMaxY();
+      } else if (_horizAxisY < enclosingBox.getMinY()) {
+        horizAxisY = enclosingBox.getMinY();
+      } else {
+        horizAxisY = _horizAxisY;
+      }
+      //assert enclosingBox.relate_yRange(horizAxis,horizAxis,ctx).intersects();
+    }
+
+  }
+
+  @Override
+  protected double getYAxis() {
+    return horizAxisY;
+  }
+
+  /**
+   * Called after bounding box is intersected.
+   * @bboxSect INTERSECTS or CONTAINS from enclosingBox's intersection
+   * @result DISJOINT, CONTAINS, or INTERSECTS (not WITHIN)
+   */
+  @Override
+  protected SpatialRelation relateRectanglePhase2(Rectangle r, SpatialRelation bboxSect, SpatialContext ctx) {
+
+    //Rectangle wraps around the world longitudinally creating a solid band; there are no corners to test intersection
+    if (r.getWidth() == 360) {
+      return SpatialRelation.INTERSECTS;
+    }
+
+    if (inverseCircle != null) {
+      return inverseCircle.relate(r, ctx).inverse();
+    }
+
+    //if a pole is wrapped, we have a separate algorithm
+    if (enclosingBox.getWidth() == 360) {
+      return relateRectangleCircleWrapsPole(r, ctx);
+    }
+
+    //This is an optimization path for when there are no dateline or pole issues.
+    if (!enclosingBox.getCrossesDateLine() && !r.getCrossesDateLine()) {
+      return super.relateRectanglePhase2(r, bboxSect, ctx);
+    }
+
+    //do quick check to see if all corners are within this circle for CONTAINS
+    int cornersIntersect = numCornersIntersect(r);
+    if (cornersIntersect == 4) {
+      //ensure r's x axis is within c's.  If it doesn't, r sneaks around the globe to touch the other side (intersect).
+      SpatialRelation xIntersect = r.relate_xRange(enclosingBox.getMinX(), enclosingBox.getMaxX(), ctx);
+      if (xIntersect == SpatialRelation.WITHIN)
+        return SpatialRelation.CONTAINS;
+      return SpatialRelation.INTERSECTS;
+    }
+
+    //INTERSECT or DISJOINT ?
+    if (cornersIntersect > 0)
+      return SpatialRelation.INTERSECTS;
+
+    //Now we check if one of the axis of the circle intersect with r.  If so we have
+    // intersection.
+
+    /* x axis intersects  */
+    if ( r.relate_yRange(getYAxis(), getYAxis(), ctx).intersects() // at y vertical
+          && r.relate_xRange(enclosingBox.getMinX(), enclosingBox.getMaxX(), ctx).intersects() )
+      return SpatialRelation.INTERSECTS;
+
+    /* y axis intersects */
+    if (r.relate_xRange(getXAxis(), getXAxis(), ctx).intersects()) { // at x horizontal
+      double yTop = getCenter().getY()+ distDEG;
+      assert yTop <= 90;
+      double yBot = getCenter().getY()- distDEG;
+      assert yBot >= -90;
+      if (r.relate_yRange(yBot, yTop, ctx).intersects())//back bottom
+        return SpatialRelation.INTERSECTS;
+    }
+
+    return SpatialRelation.DISJOINT;
+  }
+
+  private SpatialRelation relateRectangleCircleWrapsPole(Rectangle r, SpatialContext ctx) {
+    //This method handles the case where the circle wraps ONE pole, but not both.  For both,
+    // there is the inverseCircle case handled before now.  The only exception is for the case where
+    // the circle covers the entire globe, and we'll check that first.
+    if (distDEG == 180)//whole globe
+      return SpatialRelation.CONTAINS;
+
+    //Check if r is within the pole wrap region:
+    double yTop = getCenter().getY()+ distDEG;
+    if (yTop > 90) {
+      double yTopOverlap = yTop - 90;
+      assert yTopOverlap <= 90;
+      if (r.getMinY() >= 90 - yTopOverlap)
+        return SpatialRelation.CONTAINS;
+    } else {
+      double yBot = point.getY() - distDEG;
+      if (yBot < -90) {
+        double yBotOverlap = -90 - yBot;
+        assert yBotOverlap <= 90;
+        if (r.getMaxY() <= -90 + yBotOverlap)
+          return SpatialRelation.CONTAINS;
+      } else {
+        //This point is probably not reachable ??
+        assert yTop == 90 || yBot == -90;//we simply touch a pole
+        //continue
+      }
+    }
+
+    //If there are no corners to check intersection because r wraps completely...
+    if (r.getWidth() == 360)
+      return SpatialRelation.INTERSECTS;
+
+    //Check corners:
+    int cornersIntersect = numCornersIntersect(r);
+    // (It might be possible to reduce contains() calls within nCI() to exactly two, but this intersection
+    //  code is complicated enough as it is.)
+    if (cornersIntersect == 4) {//all
+      double backX = ctx.normX(getCenter().getX()+180);
+      if (r.relate_xRange(backX, backX, ctx).intersects())
+        return SpatialRelation.INTERSECTS;
+      else
+        return SpatialRelation.CONTAINS;
+    } else if (cornersIntersect == 0) {//none
+      double frontX = getCenter().getX();
+      if (r.relate_xRange(frontX, frontX, ctx).intersects())
+        return SpatialRelation.INTERSECTS;
+      else
+        return SpatialRelation.DISJOINT;
+    } else//partial
+      return SpatialRelation.INTERSECTS;
+  }
+
+  /** Returns either 0 for none, 1 for some, or 4 for all. */
+  private int numCornersIntersect(Rectangle r) {
+    //We play some logic games to avoid calling contains() which can be expensive.
+    boolean bool;//if true then all corners intersect, if false then no corners intersect
+    // for partial, we exit early with 1 and ignore bool.
+    bool = (contains(r.getMinX(),r.getMinY()));
+    if (contains(r.getMinX(),r.getMaxY())) {
+      if (!bool)
+        return 1;//partial
+    } else {
+      if (bool)
+        return 1;//partial
+    }
+    if (contains(r.getMaxX(),r.getMinY())) {
+      if (!bool)
+        return 1;//partial
+    } else {
+      if (bool)
+        return 1;//partial
+    }
+    if (contains(r.getMaxX(),r.getMaxY())) {
+      if (!bool)
+        return 1;//partial
+    } else {
+      if (bool)
+        return 1;//partial
+    }
+    return bool?4:0;
+  }
+
+  @Override
+  public String toString() {
+    //I'm deliberately making this look basic and not fully detailed with class name & misc fields.
+    //Add distance in degrees, which is easier to recognize, and earth radius agnostic.
+    String dStr = String.format("%.1f",distance);
+    if (ctx.isGeo()) {
+      double distDEG = ctx.getDistCalc().distanceToDegrees(distance);
+      dStr += String.format("=%.1f\u00B0",distDEG);
+    }
+    return "Circle(" + point + ",d=" + dStr + ')';
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/simple/PointImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/simple/PointImpl.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/simple/PointImpl.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/simple/PointImpl.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,96 @@
+/*
+ * 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.base.shape.simple;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.lucene.spatial.base.shape.SpatialRelation;
+import org.apache.lucene.spatial.base.context.SpatialContext;
+import org.apache.lucene.spatial.base.shape.Rectangle;
+import org.apache.lucene.spatial.base.shape.Point;
+import org.apache.lucene.spatial.base.shape.Shape;
+
+
+public class PointImpl implements Point {
+
+  private final double x;
+  private final double y;
+
+  public PointImpl(double x, double y) {
+    this.x = x;
+    this.y = y;
+  }
+
+  @Override
+  public double getX() {
+    return x;
+  }
+
+  @Override
+  public double getY() {
+    return y;
+  }
+  @Override
+  public Rectangle getBoundingBox() {
+    return new RectangleImpl(x, x, y, y);
+  }
+
+  @Override
+  public PointImpl getCenter() {
+    return this;
+  }
+
+  @Override
+  public SpatialRelation relate(Shape other, SpatialContext ctx) {
+    if (other instanceof Point)
+      return this.equals(other) ? SpatialRelation.INTERSECTS : SpatialRelation.DISJOINT;
+    return other.relate(this, ctx).transpose();
+  }
+
+  @Override
+  public boolean hasArea() {
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return "Pt(x="+x+",y="+y+")";
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null) { return false; }
+    if (obj == this) { return true; }
+    if (obj.getClass() != getClass()) {
+      return false;
+    }
+    PointImpl rhs = (PointImpl) obj;
+    return new EqualsBuilder()
+                  .append(x, rhs.x)
+                  .append(y, rhs.y)
+                  .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(5, 89).
+      append(x).
+      append(y).
+      toHashCode();
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/simple/RectangleImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/simple/RectangleImpl.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/simple/RectangleImpl.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/base/shape/simple/RectangleImpl.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,248 @@
+/*
+ * 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.base.shape.simple;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.lucene.spatial.base.shape.SpatialRelation;
+import org.apache.lucene.spatial.base.context.SpatialContext;
+import org.apache.lucene.spatial.base.distance.DistanceUtils;
+import org.apache.lucene.spatial.base.shape.*;
+
+/**
+ * A simple Rectangle implementation that also supports a longitudinal wrap-around. When minX > maxX, this will assume
+ * it is world coordinates that cross the date line using degrees.
+ * Immutable & threadsafe.
+ */
+public class RectangleImpl implements Rectangle {
+
+  private final double minX;
+  private final double maxX;
+  private final double minY;
+  private final double maxY;
+
+  //TODO change to West South East North to be more consistent with OGC?
+  public RectangleImpl(double minX, double maxX, double minY, double maxY) {
+    //We assume any normalization / validation of params already occurred.
+    this.minX = minX;
+    this.maxX = maxX;
+    this.minY = minY;
+    this.maxY = maxY;
+    assert minY <= maxY;
+  }
+
+  /** Copy constructor. */
+  public RectangleImpl(Rectangle r) {
+    this(r.getMinX(),r.getMaxX(),r.getMinY(),r.getMaxY());
+  }
+
+  @Override
+  public boolean hasArea() {
+    return maxX != minX && maxY != minY;
+  }
+
+  @Override
+  public double getArea() {
+    return getWidth() * getHeight();
+  }
+
+  @Override
+  public boolean getCrossesDateLine() {
+    return (minX > maxX);
+  }
+
+  @Override
+  public double getHeight() {
+    return maxY - minY;
+  }
+
+  @Override
+  public double getWidth() {
+    double w = maxX - minX;
+    if (w < 0) {//only true when minX > maxX (WGS84 assumed)
+      w += 360;
+      assert w >= 0;
+    }
+    return w;
+  }
+
+  @Override
+  public double getMaxX() {
+    return maxX;
+  }
+
+  @Override
+  public double getMaxY() {
+    return maxY;
+  }
+
+  @Override
+  public double getMinX() {
+    return minX;
+  }
+
+  @Override
+  public double getMinY() {
+    return minY;
+  }
+
+  @Override
+  public Rectangle getBoundingBox() {
+    return this;
+  }
+
+  @Override
+  public SpatialRelation relate(Shape other, SpatialContext ctx) {
+    if (other instanceof Point) {
+      return relate((Point) other, ctx);
+    }
+    if (other instanceof Rectangle) {
+      return relate((Rectangle) other, ctx);
+    }
+    return other.relate(this, ctx).transpose();
+  }
+
+  public SpatialRelation relate(Point point, SpatialContext ctx) {
+    if (point.getY() > getMaxY() || point.getY() < getMinY() ||
+        (getCrossesDateLine() ?
+            (point.getX() < minX && point.getX() > maxX)
+            : (point.getX() < minX || point.getX() > maxX) ))
+      return SpatialRelation.DISJOINT;
+    return SpatialRelation.CONTAINS;
+  }
+
+  public SpatialRelation relate(Rectangle rect, SpatialContext ctx) {
+    SpatialRelation yIntersect = relate_yRange(rect.getMinY(), rect.getMaxY(), ctx);
+    if (yIntersect == SpatialRelation.DISJOINT)
+      return SpatialRelation.DISJOINT;
+
+    SpatialRelation xIntersect = relate_xRange(rect.getMinX(), rect.getMaxX(), ctx);
+    if (xIntersect == SpatialRelation.DISJOINT)
+      return SpatialRelation.DISJOINT;
+
+    if (xIntersect == yIntersect)//in agreement
+      return xIntersect;
+
+    //if one side is equal, return the other
+    if (getMinX() == rect.getMinX() && getMaxX() == rect.getMaxX())
+      return yIntersect;
+    if (getMinY() == rect.getMinY() && getMaxY() == rect.getMaxY())
+      return xIntersect;
+
+    return SpatialRelation.INTERSECTS;
+  }
+
+  public SpatialRelation relate_yRange(double ext_minY, double ext_maxY, SpatialContext ctx) {
+    if (ext_minY > maxY || ext_maxY < minY) {
+      return SpatialRelation.DISJOINT;
+    }
+
+    if (ext_minY >= minY && ext_maxY <= maxY) {
+      return SpatialRelation.CONTAINS;
+    }
+
+    if (ext_minY <= minY && ext_maxY >= maxY) {
+      return SpatialRelation.WITHIN;
+    }
+    return SpatialRelation.INTERSECTS;
+  }
+
+  @Override
+  public SpatialRelation relate_xRange(double ext_minX, double ext_maxX, SpatialContext ctx) {
+    //For ext & this we have local minX and maxX variable pairs. We rotate them so that minX <= maxX
+    double minX = this.minX;
+    double maxX = this.maxX;
+    if (ctx.isGeo()) {
+      //the 360 check is an edge-case for complete world-wrap
+      double ext_width = ext_maxX - ext_minX;
+      if (ext_width < 0)//this logic unfortunately duplicates getWidth()
+        ext_width += 360;
+
+      if (ext_width < 360) {
+        ext_maxX = ext_minX + ext_width;
+      } else {
+        ext_maxX = 180+360;
+      }
+
+      if (getWidth() < 360) {
+        maxX = minX + getWidth();
+      } else {
+        maxX = 180+360;
+      }
+
+      if (maxX < ext_minX) {
+        minX += 360;
+        maxX += 360;
+      } else if (ext_maxX < minX) {
+        ext_minX += 360;
+        ext_maxX += 360;
+      }
+    }
+
+    if (ext_minX > maxX || ext_maxX < minX ) {
+      return SpatialRelation.DISJOINT;
+    }
+
+    if (ext_minX >= minX && ext_maxX <= maxX ) {
+      return SpatialRelation.CONTAINS;
+    }
+
+    if (ext_minX <= minX && ext_maxX >= maxX ) {
+      return SpatialRelation.WITHIN;
+    }
+    return SpatialRelation.INTERSECTS;
+  }
+
+  @Override
+  public String toString() {
+    return "Rect(minX=" + minX + ",maxX=" + maxX + ",minY=" + minY + ",maxY=" + maxY + ")";
+  }
+
+  @Override
+  public Point getCenter() {
+    final double y = getHeight() / 2 + minY;
+    double x = getWidth() / 2 + minX;
+    if (minX > maxX)//WGS84
+      x = DistanceUtils.normLonDEG(x);
+    return new PointImpl(x, y);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null) { return false; }
+    if (obj == this) { return true; }
+    if (obj.getClass() != getClass()) {
+      return false;
+    }
+    RectangleImpl rhs = (RectangleImpl) obj;
+    return new EqualsBuilder()
+                  .append(minX, rhs.minX)
+                  .append(minY, rhs.minY)
+                  .append(maxX, rhs.maxX)
+                  .append(maxY, rhs.maxY)
+                  .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(41, 37).
+    append(minX).append(minY).
+    append(maxX).append(maxY).
+      toHashCode();
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/benchmark/IndexShapeTask.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/benchmark/IndexShapeTask.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/benchmark/IndexShapeTask.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/benchmark/IndexShapeTask.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,71 @@
+/*
+ * 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.benchmark;
+
+import org.apache.lucene.benchmark.byTask.PerfRunData;
+import org.apache.lucene.benchmark.byTask.tasks.PerfTask;
+import org.apache.lucene.benchmark.byTask.utils.Config;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.spatial.base.shape.Shape;
+import org.apache.lucene.spatial.strategy.SpatialFieldInfo;
+import org.apache.lucene.spatial.strategy.SpatialStrategy;
+
+import java.util.UUID;
+
+public abstract class IndexShapeTask<T extends SpatialFieldInfo> extends PerfTask implements StrategyAware<T> {
+
+  private ShapeGenerator shapeGenerator;
+  private int numShapes;
+
+  public IndexShapeTask(PerfRunData runData) {
+    super(runData);
+  }
+
+  @Override
+  public void setup() throws Exception {
+    Config config = getRunData().getConfig();
+    String shapeGeneratorName = config.get("index.shapegenerator", ""); // TODO (cmale) - Setup default shape generator
+    shapeGenerator = (ShapeGenerator) Class.forName(shapeGeneratorName)
+            .getConstructor(Config.class)
+            .newInstance(config);
+    numShapes = config.get("index.numshapes", 1);
+  }
+
+  @Override
+  public int doLogic() throws Exception {
+    SpatialStrategy<T> spatialStrategy = createSpatialStrategy();
+    T fieldInfo = createFieldInfo();
+    for (int i = 0; i < numShapes; i++) {
+      Shape shape = shapeGenerator.generate();
+      IndexableField[] fields = spatialStrategy.createFields(fieldInfo, shape, true, true);
+      if (fields == null) {
+        continue;
+      }
+      Document document = new Document();
+      document.add(new Field("id",UUID.randomUUID().toString(),StringField.TYPE_STORED));
+      for (IndexableField field : fields) {
+        document.add(field);
+      }
+      getRunData().getIndexWriter().addDocument(document);
+    }
+    return 1;
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/benchmark/QueryShapeTask.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/benchmark/QueryShapeTask.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/benchmark/QueryShapeTask.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/benchmark/QueryShapeTask.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,52 @@
+/*
+ * 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.benchmark;
+
+import org.apache.lucene.benchmark.byTask.PerfRunData;
+import org.apache.lucene.benchmark.byTask.tasks.PerfTask;
+import org.apache.lucene.benchmark.byTask.utils.Config;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.spatial.base.query.SpatialArgs;
+import org.apache.lucene.spatial.base.query.SpatialArgsParser;
+import org.apache.lucene.spatial.strategy.SpatialFieldInfo;
+
+
+public abstract class QueryShapeTask<T extends SpatialFieldInfo> extends PerfTask implements StrategyAware<T> {
+
+  private SpatialArgs spatialArgs;
+
+  public QueryShapeTask(PerfRunData runData) {
+    super(runData);
+  }
+
+  @Override
+  public void setup() {
+    Config config = getRunData().getConfig();
+    String rawQuery = config.get("query.shapequery", ""); // TODO (cmale) - Come up with default query
+    this.spatialArgs = new SpatialArgsParser().parse(rawQuery, getSpatialContext());
+  }
+
+  @Override
+  public int doLogic() throws Exception {
+    Query query = createSpatialStrategy().makeQuery(spatialArgs, createFieldInfo());
+    TopDocs topDocs = getRunData().getIndexSearcher().search(query, 10);
+    System.out.println("Numfound: " + topDocs.totalHits);
+    return 1;
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/benchmark/ShapeGenerator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/benchmark/ShapeGenerator.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/benchmark/ShapeGenerator.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/benchmark/ShapeGenerator.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,37 @@
+/*
+ * 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.benchmark;
+
+import org.apache.lucene.benchmark.byTask.utils.Config;
+import org.apache.lucene.spatial.base.shape.Shape;
+
+
+public abstract class ShapeGenerator {
+
+  private Config config;
+
+  protected ShapeGenerator(Config config) {
+    this.config = config;
+  }
+
+  public abstract Shape generate();
+
+  protected Config getConfig() {
+    return config;
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/benchmark/StrategyAware.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/benchmark/StrategyAware.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/benchmark/StrategyAware.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/benchmark/StrategyAware.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,32 @@
+/*
+ * 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.benchmark;
+
+import org.apache.lucene.spatial.base.context.SpatialContext;
+import org.apache.lucene.spatial.strategy.SpatialFieldInfo;
+import org.apache.lucene.spatial.strategy.SpatialStrategy;
+
+
+public interface StrategyAware<T extends SpatialFieldInfo> {
+
+  T createFieldInfo();
+
+  SpatialStrategy<T> createSpatialStrategy();
+
+  SpatialContext getSpatialContext();
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/SimpleSpatialFieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/SimpleSpatialFieldInfo.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/SimpleSpatialFieldInfo.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/SimpleSpatialFieldInfo.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,32 @@
+/*
+ * 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.strategy;
+
+
+public class SimpleSpatialFieldInfo implements SpatialFieldInfo {
+
+  private final String fieldName;
+
+  public SimpleSpatialFieldInfo(String fieldName) {
+    this.fieldName = fieldName;
+  }
+
+  public String getFieldName() {
+    return fieldName;
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/SpatialFieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/SpatialFieldInfo.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/SpatialFieldInfo.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/SpatialFieldInfo.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,24 @@
+/*
+ * 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.strategy;
+
+/**
+ * Information the strategy needs for the lucene fields
+ */
+public interface SpatialFieldInfo {
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/SpatialStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/SpatialStrategy.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/SpatialStrategy.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/SpatialStrategy.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,82 @@
+/*
+ * 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.strategy;
+
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.spatial.base.context.SpatialContext;
+import org.apache.lucene.spatial.base.query.SpatialArgs;
+import org.apache.lucene.spatial.base.shape.Shape;
+
+/**
+ * must be thread safe
+ */
+public abstract class SpatialStrategy<T extends SpatialFieldInfo> {
+
+  protected boolean ignoreIncompatibleGeometry = false;
+  protected final SpatialContext ctx;
+
+  public SpatialStrategy(SpatialContext ctx) {
+    this.ctx = ctx;
+  }
+
+  public SpatialContext getSpatialContext() {
+    return ctx;
+  }
+
+  /** Corresponds with Solr's  FieldType.isPolyField(). */
+  public boolean isPolyField() {
+    return false;
+  }
+
+  /**
+   * Corresponds with Solr's FieldType.createField().
+   *
+   * This may return a null field if it does not want to make anything.
+   * This is reasonable behavior if 'ignoreIncompatibleGeometry=true' and the
+   * geometry is incompatible
+   */
+  public abstract IndexableField createField(T fieldInfo, Shape shape, boolean index, boolean store);
+
+  /** Corresponds with Solr's FieldType.createFields(). */
+  public IndexableField[] createFields(T fieldInfo, Shape shape, boolean index, boolean store) {
+    return new IndexableField[] { createField(fieldInfo, shape, index, store) };
+  }
+
+  public abstract ValueSource makeValueSource(SpatialArgs args, T fieldInfo);
+
+  /**
+   * Make a query
+   */
+  public abstract Query makeQuery(SpatialArgs args, T fieldInfo);
+
+  /**
+   * Make a Filter
+   */
+  public abstract Filter makeFilter(SpatialArgs args, T fieldInfo);
+
+  public boolean isIgnoreIncompatibleGeometry() {
+    return ignoreIncompatibleGeometry;
+  }
+
+  public void setIgnoreIncompatibleGeometry(boolean ignoreIncompatibleGeometry) {
+    this.ignoreIncompatibleGeometry = ignoreIncompatibleGeometry;
+  }
+}