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/04 07:08:29 UTC

[27/34] lucene-solr git commit: LUCENE-6997: refactors lucene-spatial module to a new lucene-spatial-extras module, and refactors sandbox GeoPointField and queries to lucene-spatial module

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java
new file mode 100644
index 0000000..dd9aa1f
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java
@@ -0,0 +1,100 @@
+/*
+ * 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.prefix.tree;
+
+import java.util.Map;
+
+import com.spatial4j.core.context.SpatialContext;
+import com.spatial4j.core.distance.DistanceUtils;
+
+/**
+ * Abstract Factory for creating {@link SpatialPrefixTree} instances with useful
+ * defaults and passed on configurations defined in a Map.
+ *
+ * @lucene.experimental
+ */
+public abstract class SpatialPrefixTreeFactory {
+
+  private static final double DEFAULT_GEO_MAX_DETAIL_KM = 0.001;//1m
+  public static final String PREFIX_TREE = "prefixTree";
+  public static final String MAX_LEVELS = "maxLevels";
+  public static final String MAX_DIST_ERR = "maxDistErr";
+
+  protected Map<String, String> args;
+  protected SpatialContext ctx;
+  protected Integer maxLevels;
+
+  /**
+   * The factory  is looked up via "prefixTree" in args, expecting "geohash" or "quad".
+   * If it's neither of these, then "geohash" is chosen for a geo context, otherwise "quad" is chosen.
+   */
+  public static SpatialPrefixTree makeSPT(Map<String,String> args, ClassLoader classLoader, SpatialContext ctx) {
+    SpatialPrefixTreeFactory instance;
+    String cname = args.get(PREFIX_TREE);
+    if (cname == null)
+      cname = ctx.isGeo() ? "geohash" : "quad";
+    if ("geohash".equalsIgnoreCase(cname))
+      instance = new GeohashPrefixTree.Factory();
+    else if ("quad".equalsIgnoreCase(cname))
+      instance = new QuadPrefixTree.Factory();
+    else if ("packedQuad".equalsIgnoreCase(cname))
+      instance = new PackedQuadPrefixTree.Factory();
+    else {
+      try {
+        Class<?> c = classLoader.loadClass(cname);
+        instance = (SpatialPrefixTreeFactory) c.newInstance();
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+    instance.init(args,ctx);
+    return instance.newSPT();
+  }
+
+  protected void init(Map<String, String> args, SpatialContext ctx) {
+    this.args = args;
+    this.ctx = ctx;
+    initMaxLevels();
+  }
+
+  protected void initMaxLevels() {
+    String mlStr = args.get(MAX_LEVELS);
+    if (mlStr != null) {
+      maxLevels = Integer.valueOf(mlStr);
+      return;
+    }
+
+    double degrees;
+    String maxDetailDistStr = args.get(MAX_DIST_ERR);
+    if (maxDetailDistStr == null) {
+      if (!ctx.isGeo()) {
+        return;//let default to max
+      }
+      degrees = DistanceUtils.dist2Degrees(DEFAULT_GEO_MAX_DETAIL_KM, DistanceUtils.EARTH_MEAN_RADIUS_KM);
+    } else {
+      degrees = Double.parseDouble(maxDetailDistStr);
+    }
+    maxLevels = getLevelForDistance(degrees);
+  }
+
+  /** Calls {@link SpatialPrefixTree#getLevelForDistance(double)}. */
+  protected abstract int getLevelForDistance(double degrees);
+
+  protected abstract SpatialPrefixTree newSPT();
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/TreeCellIterator.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/TreeCellIterator.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/TreeCellIterator.java
new file mode 100644
index 0000000..f5e4a76
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/TreeCellIterator.java
@@ -0,0 +1,88 @@
+package org.apache.lucene.spatial.prefix.tree;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import com.spatial4j.core.shape.Point;
+import com.spatial4j.core.shape.Shape;
+
+/**
+ * Navigates a {@link org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree} from a given cell (typically the world
+ * cell) down to a maximum number of configured levels, filtered by a given shape. Intermediate non-leaf cells are
+ * returned. It supports {@link #remove()} for skipping traversal of subcells of the current cell.
+ *
+ * @lucene.internal
+ */
+class TreeCellIterator extends CellIterator {
+  //This class uses a stack approach, which is more efficient than creating linked nodes. And it might more easily
+  // pave the way for re-using Cell & CellIterator at a given level in the future.
+
+  private final Shape shapeFilter;//possibly null
+  private final CellIterator[] iterStack;//starts at level 1
+  private int stackIdx;//-1 when done
+  private boolean descend;
+
+  public TreeCellIterator(Shape shapeFilter, int detailLevel, Cell parentCell) {
+    this.shapeFilter = shapeFilter;
+    assert parentCell.getLevel() == 0;
+    iterStack = new CellIterator[detailLevel];
+    iterStack[0] = parentCell.getNextLevelCells(shapeFilter);
+    stackIdx = 0;//always points to an iter (non-null)
+    //note: not obvious but needed to visit the first cell before trying to descend
+    descend = false;
+  }
+
+  @Override
+  public boolean hasNext() {
+    if (nextCell != null)
+      return true;
+    while (true) {
+      if (stackIdx == -1)//the only condition in which we return false
+        return false;
+      //If we can descend...
+      if (descend && !(stackIdx == iterStack.length - 1 || iterStack[stackIdx].thisCell().isLeaf())) {
+        CellIterator nextIter = iterStack[stackIdx].thisCell().getNextLevelCells(shapeFilter);
+        //push stack
+        iterStack[++stackIdx] = nextIter;
+      }
+      //Get sibling...
+      if (iterStack[stackIdx].hasNext()) {
+        nextCell = iterStack[stackIdx].next();
+        //at detailLevel
+        if (stackIdx == iterStack.length - 1 && !(shapeFilter instanceof Point)) //point check is a kludge
+          nextCell.setLeaf();//because at bottom
+        break;
+      }
+      //Couldn't get next; go up...
+      //pop stack
+      iterStack[stackIdx--] = null;
+      descend = false;//so that we don't re-descend where we just were
+    }
+    assert nextCell != null;
+    descend = true;//reset
+    return true;
+  }
+
+  @Override
+  public void remove() {
+    assert thisCell() != null && nextCell == null;
+    descend = false;
+  }
+
+  //TODO implement a smart nextFrom() that looks at the parent's bytes first
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/package-info.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/package-info.java
new file mode 100644
index 0000000..4dd85b1
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/package-info.java
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+ 
+/** 
+ * This package is about SpatialPrefixTree and any supporting classes.
+ * A SpatialPrefixTree supports spatial indexing by index-time tokens
+ * where adding characters to a string gives greater resolution.
+ * <p>
+ * Potential Implementations include:
+ * <ul>
+ * <li>http://en.wikipedia.org/wiki/Quadtree
+ * <li>http://en.wikipedia.org/wiki/Geohash
+ * <li>http://healpix.jpl.nasa.gov/
+ * </ul>
+ */
+package org.apache.lucene.spatial.prefix.tree;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialArgs.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialArgs.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialArgs.java
new file mode 100644
index 0000000..5501d5c
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialArgs.java
@@ -0,0 +1,149 @@
+package org.apache.lucene.spatial.query;
+
+/*
+ * 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.
+ */
+
+import com.spatial4j.core.context.SpatialContext;
+import com.spatial4j.core.shape.Point;
+import com.spatial4j.core.shape.Rectangle;
+import com.spatial4j.core.shape.Shape;
+
+/**
+ * Principally holds the query {@link Shape} and the {@link SpatialOperation}.
+ * It's used as an argument to some methods on {@link org.apache.lucene.spatial.SpatialStrategy}.
+ *
+ * @lucene.experimental
+ */
+public class SpatialArgs {
+
+  public static final double DEFAULT_DISTERRPCT = 0.025d;
+
+  private SpatialOperation operation;
+  private Shape shape;
+  private Double distErrPct;
+  private Double distErr;
+
+  public SpatialArgs(SpatialOperation operation, Shape shape) {
+    if (operation == null || shape == null)
+      throw new NullPointerException("operation and shape are required");
+    this.operation = operation;
+    this.shape = shape;
+  }
+
+  /**
+   * Computes the distance given a shape and the {@code distErrPct}.  The
+   * algorithm is the fraction of the distance from the center of the query
+   * shape to its closest bounding box corner.
+   *
+   * @param shape Mandatory.
+   * @param distErrPct 0 to 0.5
+   * @param ctx Mandatory
+   * @return A distance (in degrees).
+   */
+  public static double calcDistanceFromErrPct(Shape shape, double distErrPct, SpatialContext ctx) {
+    if (distErrPct < 0 || distErrPct > 0.5) {
+      throw new IllegalArgumentException("distErrPct " + distErrPct + " must be between [0 to 0.5]");
+    }
+    if (distErrPct == 0 || shape instanceof Point) {
+      return 0;
+    }
+    Rectangle bbox = shape.getBoundingBox();
+    //Compute the distance from the center to a corner.  Because the distance
+    // to a bottom corner vs a top corner can vary in a geospatial scenario,
+    // take the closest one (greater precision).
+    Point ctr = bbox.getCenter();
+    double y = (ctr.getY() >= 0 ? bbox.getMaxY() : bbox.getMinY());
+    double diagonalDist = ctx.getDistCalc().distance(ctr, bbox.getMaxX(), y);
+    return diagonalDist * distErrPct;
+  }
+
+  /**
+   * Gets the error distance that specifies how precise the query shape is. This
+   * looks at {@link #getDistErr()}, {@link #getDistErrPct()}, and {@code
+   * defaultDistErrPct}.
+   * @param defaultDistErrPct 0 to 0.5
+   * @return {@code >= 0}
+   */
+  public double resolveDistErr(SpatialContext ctx, double defaultDistErrPct) {
+    if (distErr != null)
+      return distErr;
+    double distErrPct = (this.distErrPct != null ? this.distErrPct : defaultDistErrPct);
+    return calcDistanceFromErrPct(shape, distErrPct, ctx);
+  }
+
+  /** Check if the arguments make sense -- throw an exception if not */
+  public void validate() throws IllegalArgumentException {
+    if (distErr != null && distErrPct != null)
+      throw new IllegalArgumentException("Only distErr or distErrPct can be specified.");
+  }
+
+  @Override
+  public String toString() {
+    return SpatialArgsParser.writeSpatialArgs(this);
+  }
+
+  //------------------------------------------------
+  // Getters & Setters
+  //------------------------------------------------
+
+  public SpatialOperation getOperation() {
+    return operation;
+  }
+
+  public void setOperation(SpatialOperation operation) {
+    this.operation = operation;
+  }
+
+  public Shape getShape() {
+    return shape;
+  }
+
+  public void setShape(Shape shape) {
+    this.shape = shape;
+  }
+
+  /**
+   * A measure of acceptable error of the shape as a fraction.  This effectively
+   * inflates the size of the shape but should not shrink it.
+   *
+   * @return 0 to 0.5
+   * @see #calcDistanceFromErrPct(com.spatial4j.core.shape.Shape, double,
+   *      com.spatial4j.core.context.SpatialContext)
+   */
+  public Double getDistErrPct() {
+    return distErrPct;
+  }
+
+  public void setDistErrPct(Double distErrPct) {
+    if (distErrPct != null)
+      this.distErrPct = distErrPct;
+  }
+
+  /**
+   * The acceptable error of the shape.  This effectively inflates the
+   * size of the shape but should not shrink it.
+   *
+   * @return {@code >= 0}
+   */
+  public Double getDistErr() {
+    return distErr;
+  }
+
+  public void setDistErr(Double distErr) {
+    this.distErr = distErr;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialArgsParser.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialArgsParser.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialArgsParser.java
new file mode 100644
index 0000000..6e20814
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialArgsParser.java
@@ -0,0 +1,147 @@
+package org.apache.lucene.spatial.query;
+
+/*
+ * 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.
+ */
+
+import com.spatial4j.core.context.SpatialContext;
+import com.spatial4j.core.exception.InvalidShapeException;
+import com.spatial4j.core.shape.Shape;
+
+import java.text.ParseException;
+import java.util.HashMap;
+import java.util.Locale;
+import java.util.Map;
+import java.util.StringTokenizer;
+
+/**
+ * Parses a string that usually looks like "OPERATION(SHAPE)" into a {@link SpatialArgs}
+ * object. The set of operations supported are defined in {@link SpatialOperation}, such
+ * as "Intersects" being a common one. The shape portion is defined by WKT {@link com.spatial4j.core.io.WktShapeParser},
+ * but it can be overridden/customized via {@link #parseShape(String, com.spatial4j.core.context.SpatialContext)}.
+ * There are some optional name-value pair parameters that follow the closing parenthesis.  Example:
+ * <pre>
+ *   Intersects(ENVELOPE(-10,-8,22,20)) distErrPct=0.025
+ * </pre>
+ * <p>
+ * In the future it would be good to support something at least semi-standardized like a
+ * variant of <a href="http://docs.geoserver.org/latest/en/user/filter/ecql_reference.html#spatial-predicate">
+ *   [E]CQL</a>.
+ *
+ * @lucene.experimental
+ */
+public class SpatialArgsParser {
+
+  public static final String DIST_ERR_PCT = "distErrPct";
+  public static final String DIST_ERR = "distErr";
+
+  /** Writes a close approximation to the parsed input format. */
+  static String writeSpatialArgs(SpatialArgs args) {
+    StringBuilder str = new StringBuilder();
+    str.append(args.getOperation().getName());
+    str.append('(');
+    str.append(args.getShape().toString());
+    if (args.getDistErrPct() != null)
+      str.append(" distErrPct=").append(String.format(Locale.ROOT, "%.2f%%", args.getDistErrPct() * 100d));
+    if (args.getDistErr() != null)
+      str.append(" distErr=").append(args.getDistErr());
+    str.append(')');
+    return str.toString();
+  }
+
+  /**
+   * Parses a string such as "Intersects(ENVELOPE(-10,-8,22,20)) distErrPct=0.025".
+   *
+   * @param v   The string to parse. Mandatory.
+   * @param ctx The spatial context. Mandatory.
+   * @return Not null.
+   * @throws IllegalArgumentException if the parameters don't make sense or an add-on parameter is unknown
+   * @throws ParseException If there is a problem parsing the string
+   * @throws InvalidShapeException When the coordinates are invalid for the shape
+   */
+  public SpatialArgs parse(String v, SpatialContext ctx) throws ParseException, InvalidShapeException {
+    int idx = v.indexOf('(');
+    int edx = v.lastIndexOf(')');
+
+    if (idx < 0 || idx > edx) {
+      throw new ParseException("missing parens: " + v, -1);
+    }
+
+    SpatialOperation op = SpatialOperation.get(v.substring(0, idx).trim());
+
+    String body = v.substring(idx + 1, edx).trim();
+    if (body.length() < 1) {
+      throw new ParseException("missing body : " + v, idx + 1);
+    }
+
+    Shape shape = parseShape(body, ctx);
+    SpatialArgs args = newSpatialArgs(op, shape);
+
+    if (v.length() > (edx + 1)) {
+      body = v.substring(edx + 1).trim();
+      if (body.length() > 0) {
+        Map<String, String> aa = parseMap(body);
+        readNameValuePairs(args, aa);
+        if (!aa.isEmpty()) {
+          throw new IllegalArgumentException("unused parameters: " + aa);
+        }
+      }
+    }
+    args.validate();
+    return args;
+  }
+
+  protected SpatialArgs newSpatialArgs(SpatialOperation op, Shape shape) {
+    return new SpatialArgs(op, shape);
+  }
+
+  protected void readNameValuePairs(SpatialArgs args, Map<String, String> nameValPairs) {
+    args.setDistErrPct(readDouble(nameValPairs.remove(DIST_ERR_PCT)));
+    args.setDistErr(readDouble(nameValPairs.remove(DIST_ERR)));
+  }
+
+  protected Shape parseShape(String str, SpatialContext ctx) throws ParseException {
+    //return ctx.readShape(str);//still in Spatial4j 0.4 but will be deleted
+    return ctx.readShapeFromWkt(str);
+  }
+
+  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);
+  }
+
+  /** Parses "a=b c=d f" (whitespace separated) into name-value pairs. If there
+   * is no '=' as in 'f' above then it's short for f=f. */
+  protected static Map<String, String> parseMap(String body) {
+    Map<String, String> map = new HashMap<>();
+    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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialOperation.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialOperation.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialOperation.java
new file mode 100644
index 0000000..753a064
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialOperation.java
@@ -0,0 +1,180 @@
+package org.apache.lucene.spatial.query;
+
+/*
+ * 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.
+ */
+
+import com.spatial4j.core.shape.Rectangle;
+import com.spatial4j.core.shape.Shape;
+import com.spatial4j.core.shape.SpatialRelation;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+/**
+ * A predicate that compares a stored geometry to a supplied geometry. It's enum-like. For more
+ * explanation of each predicate, consider looking at the source implementation
+ * of {@link #evaluate(com.spatial4j.core.shape.Shape, com.spatial4j.core.shape.Shape)}. It's important
+ * to be aware that Lucene-spatial makes no distinction of shape boundaries, unlike many standardized
+ * definitions. Nor does it make dimensional distinctions (e.g. line vs polygon).
+ * You can lookup a predicate by "Covers" or "Contains", for example, and you will get the
+ * same underlying predicate implementation.
+ *
+ * @see <a href="http://en.wikipedia.org/wiki/DE-9IM">DE-9IM at Wikipedia, based on OGC specs</a>
+ * @see <a href="http://edndoc.esri.com/arcsde/9.1/general_topics/understand_spatial_relations.htm">
+ *   ESRIs docs on spatial relations</a>
+ *
+ * @lucene.experimental
+ */
+public abstract class SpatialOperation implements Serializable {
+  //TODO rename to SpatialPredicate. Use enum?  LUCENE-5771
+
+  // Private registry
+  private static final Map<String, SpatialOperation> registry = new HashMap<>();//has aliases
+  private static final List<SpatialOperation> list = new ArrayList<>();
+
+  // Geometry Operations
+
+  /** Bounding box of the *indexed* shape, then {@link #Intersects}. */
+  public static final SpatialOperation BBoxIntersects = new SpatialOperation("BBoxIntersects") {
+    @Override
+    public boolean evaluate(Shape indexedShape, Shape queryShape) {
+      return indexedShape.getBoundingBox().relate(queryShape).intersects();
+    }
+  };
+  /** Bounding box of the *indexed* shape, then {@link #IsWithin}. */
+  public static final SpatialOperation BBoxWithin     = new SpatialOperation("BBoxWithin") {
+    {
+      register("BBoxCoveredBy");//alias -- the better name
+    }
+    @Override
+    public boolean evaluate(Shape indexedShape, Shape queryShape) {
+      Rectangle bbox = indexedShape.getBoundingBox();
+      return bbox.relate(queryShape) == SpatialRelation.WITHIN || bbox.equals(queryShape);
+    }
+  };
+  /** Meets the "Covers" OGC definition (boundary-neutral). */
+  public static final SpatialOperation Contains       = new SpatialOperation("Contains") {
+    {
+      register("Covers");//alias -- the better name
+    }
+    @Override
+    public boolean evaluate(Shape indexedShape, Shape queryShape) {
+      return indexedShape.relate(queryShape) == SpatialRelation.CONTAINS || indexedShape.equals(queryShape);
+    }
+  };
+  /** Meets the "Intersects" OGC definition. */
+  public static final SpatialOperation Intersects     = new SpatialOperation("Intersects") {
+    @Override
+    public boolean evaluate(Shape indexedShape, Shape queryShape) {
+      return indexedShape.relate(queryShape).intersects();
+    }
+  };
+  /** Meets the "Equals" OGC definition. */
+  public static final SpatialOperation IsEqualTo      = new SpatialOperation("Equals") {
+    {
+      register("IsEqualTo");//alias (deprecated)
+    }
+    @Override
+    public boolean evaluate(Shape indexedShape, Shape queryShape) {
+      return indexedShape.equals(queryShape);
+    }
+  };
+  /** Meets the "Disjoint" OGC definition. */
+  public static final SpatialOperation IsDisjointTo   = new SpatialOperation("Disjoint") {
+    {
+      register("IsDisjointTo");//alias (deprecated)
+    }
+    @Override
+    public boolean evaluate(Shape indexedShape, Shape queryShape) {
+      return ! indexedShape.relate(queryShape).intersects();
+    }
+  };
+  /** Meets the "CoveredBy" OGC definition (boundary-neutral). */
+  public static final SpatialOperation IsWithin       = new SpatialOperation("Within") {
+    {
+      register("IsWithin");//alias (deprecated)
+      register("CoveredBy");//alias -- the more appropriate name.
+    }
+    @Override
+    public boolean evaluate(Shape indexedShape, Shape queryShape) {
+      return indexedShape.relate(queryShape) == SpatialRelation.WITHIN || indexedShape.equals(queryShape);
+    }
+  };
+  /** Almost meets the "Overlaps" OGC definition, but boundary-neutral (boundary==interior). */
+  public static final SpatialOperation Overlaps       = new SpatialOperation("Overlaps") {
+    @Override
+    public boolean evaluate(Shape indexedShape, Shape queryShape) {
+      return indexedShape.relate(queryShape) == SpatialRelation.INTERSECTS;//not Contains or Within or Disjoint
+    }
+  };
+
+  private final String name;
+
+  protected SpatialOperation(String name) {
+    this.name = name;
+    register(name);
+    list.add( this );
+  }
+
+  protected void register(String name) {
+    registry.put(name, this);
+    registry.put(name.toUpperCase(Locale.ROOT), this);
+  }
+
+  public static SpatialOperation get( String v ) {
+    SpatialOperation op = registry.get( v );
+    if( op == null ) {
+      op = registry.get(v.toUpperCase(Locale.ROOT));
+    }
+    if( op == null ) {
+      throw new IllegalArgumentException("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;
+  }
+
+  /**
+   * Returns whether the relationship between indexedShape and queryShape is
+   * satisfied by this operation.
+   */
+  public abstract boolean evaluate(Shape indexedShape, Shape queryShape);
+
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public String toString() {
+    return name;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/UnsupportedSpatialOperation.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/UnsupportedSpatialOperation.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/UnsupportedSpatialOperation.java
new file mode 100644
index 0000000..7ac239c
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/UnsupportedSpatialOperation.java
@@ -0,0 +1,29 @@
+package org.apache.lucene.spatial.query;
+
+/*
+ * 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.
+ */
+
+/**
+ * Exception thrown when the {@link org.apache.lucene.spatial.SpatialStrategy} cannot implement the requested operation.
+ * @lucene.experimental
+ */
+public class UnsupportedSpatialOperation extends UnsupportedOperationException {
+
+  public UnsupportedSpatialOperation(SpatialOperation op) {
+    super(op.getName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/package-info.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/package-info.java
new file mode 100644
index 0000000..42a5036
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/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 Query options useful for client side requests
+ */
+package org.apache.lucene.spatial.query;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java
new file mode 100644
index 0000000..ec75a6f
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java
@@ -0,0 +1,279 @@
+package org.apache.lucene.spatial.serialized;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.FilterOutputStream;
+import java.io.IOException;
+import java.util.Map;
+
+import com.spatial4j.core.context.SpatialContext;
+import com.spatial4j.core.io.BinaryCodec;
+import com.spatial4j.core.shape.Point;
+import com.spatial4j.core.shape.Shape;
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.RandomAccessWeight;
+import org.apache.lucene.search.TwoPhaseIterator;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.spatial.SpatialStrategy;
+import org.apache.lucene.spatial.query.SpatialArgs;
+import org.apache.lucene.spatial.util.DistanceToShapeValueSource;
+import org.apache.lucene.spatial.util.ShapePredicateValueSource;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+
+
+/**
+ * A SpatialStrategy based on serializing a Shape stored into BinaryDocValues.
+ * This is not at all fast; it's designed to be used in conjunction with another index based
+ * SpatialStrategy that is approximated (like {@link org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy})
+ * to add precision or eventually make more specific / advanced calculations on the per-document
+ * geometry.
+ * The serialization uses Spatial4j's {@link com.spatial4j.core.io.BinaryCodec}.
+ *
+ * @lucene.experimental
+ */
+public class SerializedDVStrategy extends SpatialStrategy {
+
+  /**
+   * A cache heuristic for the buf size based on the last shape size.
+   */
+  //TODO do we make this non-volatile since it's merely a heuristic?
+  private volatile int indexLastBufSize = 8 * 1024;//8KB default on first run
+
+  /**
+   * Constructs the spatial strategy with its mandatory arguments.
+   */
+  public SerializedDVStrategy(SpatialContext ctx, String fieldName) {
+    super(ctx, fieldName);
+  }
+
+  @Override
+  public Field[] createIndexableFields(Shape shape) {
+    int bufSize = Math.max(128, (int) (this.indexLastBufSize * 1.5));//50% headroom over last
+    ByteArrayOutputStream byteStream = new ByteArrayOutputStream(bufSize);
+    final BytesRef bytesRef = new BytesRef();//receiver of byteStream's bytes
+    try {
+      ctx.getBinaryCodec().writeShape(new DataOutputStream(byteStream), shape);
+      //this is a hack to avoid redundant byte array copying by byteStream.toByteArray()
+      byteStream.writeTo(new FilterOutputStream(null/*not used*/) {
+        @Override
+        public void write(byte[] b, int off, int len) throws IOException {
+          bytesRef.bytes = b;
+          bytesRef.offset = off;
+          bytesRef.length = len;
+        }
+      });
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    this.indexLastBufSize = bytesRef.length;//cache heuristic
+    return new Field[]{new BinaryDocValuesField(getFieldName(), bytesRef)};
+  }
+
+  @Override
+  public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
+    //TODO if makeShapeValueSource gets lifted to the top; this could become a generic impl.
+    return new DistanceToShapeValueSource(makeShapeValueSource(), queryPoint, multiplier, ctx);
+  }
+
+  /**
+   * Returns a Query that should be used in a random-access fashion.
+   * Use in another manner will be SLOW.
+   */
+  @Override
+  public Query makeQuery(SpatialArgs args) {
+    ValueSource shapeValueSource = makeShapeValueSource();
+    ShapePredicateValueSource predicateValueSource = new ShapePredicateValueSource(
+        shapeValueSource, args.getOperation(), args.getShape());
+    return new PredicateValueSourceQuery(predicateValueSource);
+  }
+
+  /**
+   * Provides access to each shape per document as a ValueSource in which
+   * {@link org.apache.lucene.queries.function.FunctionValues#objectVal(int)} returns a {@link
+   * Shape}.
+   */ //TODO raise to SpatialStrategy
+  public ValueSource makeShapeValueSource() {
+    return new ShapeDocValueSource(getFieldName(), ctx.getBinaryCodec());
+  }
+
+  /** Warning: don't iterate over the results of this query; it's designed for use in a random-access fashion
+   * by {@link TwoPhaseIterator}.
+   */
+  static class PredicateValueSourceQuery extends Query {
+    private final ValueSource predicateValueSource;//we call boolVal(doc)
+
+    public PredicateValueSourceQuery(ValueSource predicateValueSource) {
+      this.predicateValueSource = predicateValueSource;
+    }
+
+    @Override
+    public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+      return new RandomAccessWeight(this) {
+        @Override
+        protected Bits getMatchingDocs(LeafReaderContext context) throws IOException {
+          final FunctionValues predFuncValues = predicateValueSource.getValues(null, context);
+          return new Bits() {
+            @Override
+            public boolean get(int index) {
+              return predFuncValues.boolVal(index);
+            }
+
+            @Override
+            public int length() {
+              return context.reader().maxDoc();
+            }
+          };
+        }
+      };
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (super.equals(o) == false) return false;
+
+      PredicateValueSourceQuery that = (PredicateValueSourceQuery) o;
+
+      if (!predicateValueSource.equals(that.predicateValueSource)) return false;
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return super.hashCode() + 31 * predicateValueSource.hashCode();
+    }
+    
+    @Override
+    public String toString(String field) {
+      return "PredicateValueSourceQuery(" +
+               predicateValueSource.toString() +
+             ")";
+    }
+  }//PredicateValueSourceQuery
+
+  /**
+   * Implements a ValueSource by deserializing a Shape in from BinaryDocValues using BinaryCodec.
+   * @see #makeShapeValueSource()
+   */
+  static class ShapeDocValueSource extends ValueSource {
+
+    private final String fieldName;
+    private final BinaryCodec binaryCodec;//spatial4j
+
+    private ShapeDocValueSource(String fieldName, BinaryCodec binaryCodec) {
+      this.fieldName = fieldName;
+      this.binaryCodec = binaryCodec;
+    }
+
+    @Override
+    public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
+      final BinaryDocValues docValues = readerContext.reader().getBinaryDocValues(fieldName);
+
+      return new FunctionValues() {
+        int bytesRefDoc = -1;
+        BytesRefBuilder bytesRef = new BytesRefBuilder();
+
+        boolean fillBytes(int doc) {
+          if (bytesRefDoc != doc) {
+            bytesRef.copyBytes(docValues.get(doc));
+            bytesRefDoc = doc;
+          }
+          return bytesRef.length() != 0;
+        }
+
+        @Override
+        public boolean exists(int doc) {
+          return fillBytes(doc);
+        }
+
+        @Override
+        public boolean bytesVal(int doc, BytesRefBuilder target) {
+          target.clear();
+          if (fillBytes(doc)) {
+            target.copyBytes(bytesRef);
+            return true;
+          } else {
+            return false;
+          }
+        }
+
+        @Override
+        public Object objectVal(int docId) {
+          if (!fillBytes(docId))
+            return null;
+          DataInputStream dataInput = new DataInputStream(
+              new ByteArrayInputStream(bytesRef.bytes(), 0, bytesRef.length()));
+          try {
+            return binaryCodec.readShape(dataInput);
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        }
+
+        @Override
+        public Explanation explain(int doc) {
+          return Explanation.match(Float.NaN, toString(doc));
+        }
+
+        @Override
+        public String toString(int doc) {
+          return description() + "=" + objectVal(doc);//TODO truncate?
+        }
+
+      };
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+
+      ShapeDocValueSource that = (ShapeDocValueSource) o;
+
+      if (!fieldName.equals(that.fieldName)) return false;
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int result = fieldName.hashCode();
+      return result;
+    }
+
+    @Override
+    public String description() {
+      return "shapeDocVal(" + fieldName + ")";
+    }
+  }//ShapeDocValueSource
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/package-info.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/package-info.java
new file mode 100644
index 0000000..7a316d9
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/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.
+ */
+ 
+/** 
+ * Strategies that serialize the shape (non-indexed).
+ */
+package org.apache.lucene.spatial.serialized;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShape.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShape.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShape.java
new file mode 100644
index 0000000..19118a5
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShape.java
@@ -0,0 +1,186 @@
+package org.apache.lucene.spatial.spatial4j;
+
+/*
+ * 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.
+ */
+
+import com.spatial4j.core.context.SpatialContext;
+import com.spatial4j.core.distance.DistanceUtils;
+import com.spatial4j.core.shape.Point;
+import com.spatial4j.core.shape.Rectangle;
+import com.spatial4j.core.shape.Shape;
+import com.spatial4j.core.shape.SpatialRelation;
+import com.spatial4j.core.shape.impl.RectangleImpl;
+import org.apache.lucene.geo3d.LatLonBounds;
+import org.apache.lucene.geo3d.GeoArea;
+import org.apache.lucene.geo3d.GeoAreaFactory;
+import org.apache.lucene.geo3d.GeoPoint;
+import org.apache.lucene.geo3d.GeoShape;
+import org.apache.lucene.geo3d.PlanetModel;
+
+/**
+ * A Spatial4j Shape wrapping a {@link GeoShape} ("Geo3D") -- a 3D planar geometry based Spatial4j Shape implementation.
+ * Geo3D implements shapes on the surface of a sphere or ellipsoid.
+ *
+ * @lucene.experimental
+ */
+public class Geo3dShape implements Shape {
+  /** The required size of this adjustment depends on the actual planetary model chosen.
+   * This value is big enough to account for WGS84. */
+  protected static final double ROUNDOFF_ADJUSTMENT = 0.05;
+
+  public final SpatialContext ctx;
+  public final GeoShape shape;
+  public final PlanetModel planetModel;
+
+  private volatile Rectangle boundingBox = null; // lazy initialized
+
+  public Geo3dShape(final GeoShape shape, final SpatialContext ctx) {
+    this(PlanetModel.SPHERE, shape, ctx);
+  }
+  
+  public Geo3dShape(final PlanetModel planetModel, final GeoShape shape, final SpatialContext ctx) {
+    if (!ctx.isGeo()) {
+      throw new IllegalArgumentException("SpatialContext.isGeo() must be true");
+    }
+    this.ctx = ctx;
+    this.planetModel = planetModel;
+    this.shape = shape;
+  }
+
+  @Override
+  public SpatialContext getContext() {
+    return ctx;
+  }
+
+  @Override
+  public SpatialRelation relate(Shape other) {
+    if (other instanceof Rectangle)
+      return relate((Rectangle)other);
+    else if (other instanceof Point)
+      return relate((Point)other);
+    else
+      throw new RuntimeException("Unimplemented shape relationship determination: " + other.getClass());
+  }
+
+  protected SpatialRelation relate(Rectangle r) {
+    // Construct the right kind of GeoArea first
+    GeoArea geoArea = GeoAreaFactory.makeGeoArea(planetModel,
+        r.getMaxY() * DistanceUtils.DEGREES_TO_RADIANS,
+        r.getMinY() * DistanceUtils.DEGREES_TO_RADIANS,
+        r.getMinX() * DistanceUtils.DEGREES_TO_RADIANS,
+        r.getMaxX() * DistanceUtils.DEGREES_TO_RADIANS);
+    int relationship = geoArea.getRelationship(shape);
+    if (relationship == GeoArea.WITHIN)
+      return SpatialRelation.WITHIN;
+    else if (relationship == GeoArea.CONTAINS)
+      return SpatialRelation.CONTAINS;
+    else if (relationship == GeoArea.OVERLAPS)
+      return SpatialRelation.INTERSECTS;
+    else if (relationship == GeoArea.DISJOINT)
+      return SpatialRelation.DISJOINT;
+    else
+      throw new RuntimeException("Unknown relationship returned: "+relationship);
+  }
+
+  protected SpatialRelation relate(Point p) {
+    // Create a GeoPoint
+    GeoPoint point = new GeoPoint(planetModel, p.getY()* DistanceUtils.DEGREES_TO_RADIANS, p.getX()* DistanceUtils.DEGREES_TO_RADIANS);
+    if (shape.isWithin(point)) {
+      // Point within shape
+      return SpatialRelation.CONTAINS;
+    }
+    return SpatialRelation.DISJOINT;
+  }
+
+
+  
+  @Override
+  public Rectangle getBoundingBox() {
+    Rectangle bbox = this.boundingBox;//volatile read once
+    if (bbox == null) {
+      LatLonBounds bounds = new LatLonBounds();
+      shape.getBounds(bounds);
+      double leftLon;
+      double rightLon;
+      if (bounds.checkNoLongitudeBound()) {
+        leftLon = -180.0;
+        rightLon = 180.0;
+      } else {
+        leftLon = bounds.getLeftLongitude().doubleValue() * DistanceUtils.RADIANS_TO_DEGREES;
+        rightLon = bounds.getRightLongitude().doubleValue() * DistanceUtils.RADIANS_TO_DEGREES;
+      }
+      double minLat;
+      if (bounds.checkNoBottomLatitudeBound()) {
+        minLat = -90.0;
+      } else {
+        minLat = bounds.getMinLatitude().doubleValue() * DistanceUtils.RADIANS_TO_DEGREES;
+      }
+      double maxLat;
+      if (bounds.checkNoTopLatitudeBound()) {
+        maxLat = 90.0;
+      } else {
+        maxLat = bounds.getMaxLatitude().doubleValue() * DistanceUtils.RADIANS_TO_DEGREES;
+      }
+      bbox = new RectangleImpl(leftLon, rightLon, minLat, maxLat, ctx).getBuffered(ROUNDOFF_ADJUSTMENT, ctx);
+      this.boundingBox = bbox;
+    }
+    return bbox;
+  }
+
+  @Override
+  public boolean hasArea() {
+    return true;
+  }
+
+  @Override
+  public double getArea(SpatialContext ctx) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Point getCenter() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Shape getBuffered(double distance, SpatialContext ctx) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return "Geo3dShape{planetmodel=" + planetModel + ", shape=" + shape + '}';
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof Geo3dShape))
+      return false;
+    Geo3dShape tr = (Geo3dShape)other;
+    return tr.ctx.equals(ctx) && tr.planetModel.equals(planetModel) && tr.shape.equals(shape);
+  }
+
+  @Override
+  public int hashCode() {
+    return planetModel.hashCode() + shape.hashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/package-info.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/package-info.java
new file mode 100644
index 0000000..7815318
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+
+/** Spatial4j stuff that ideally belongs in Spatial4j (isn't related to Lucene). */
+package org.apache.lucene.spatial.spatial4j;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/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..9a91782
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/CachingDoubleValueSource.java
@@ -0,0 +1,94 @@
+/*
+ * 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/b216c118/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..703586b
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java
@@ -0,0 +1,123 @@
+package org.apache.lucene.spatial.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+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/b216c118/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..b08260a
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeAreaValueSource.java
@@ -0,0 +1,117 @@
+package org.apache.lucene.spatial.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+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/b216c118/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..e26f290
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCache.java
@@ -0,0 +1,55 @@
+/*
+ * 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/b216c118/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..198b062
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java
@@ -0,0 +1,113 @@
+package org.apache.lucene.spatial.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+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/b216c118/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..a57460d
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheProvider.java
@@ -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.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/b216c118/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..018b7ca
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapePredicateValueSource.java
@@ -0,0 +1,114 @@
+package org.apache.lucene.spatial.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+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/b216c118/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..7fec149
--- /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.
+ */
+
+/**
+ * Various spatial utilities.
+ */
+package org.apache.lucene.spatial.util;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/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..85afeae
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java
@@ -0,0 +1,121 @@
+package org.apache.lucene.spatial.vector;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+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();
+  }
+}