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

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
new file mode 100644
index 0000000..9565cbf
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
@@ -0,0 +1,588 @@
+/*
+ * 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.bbox;
+
+import com.spatial4j.core.context.SpatialContext;
+import com.spatial4j.core.shape.Point;
+import com.spatial4j.core.shape.Rectangle;
+import com.spatial4j.core.shape.Shape;
+import org.apache.lucene.document.LegacyDoubleField;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.LegacyNumericRangeQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.spatial.SpatialStrategy;
+import org.apache.lucene.spatial.query.SpatialArgs;
+import org.apache.lucene.spatial.query.SpatialOperation;
+import org.apache.lucene.spatial.query.UnsupportedSpatialOperation;
+import org.apache.lucene.spatial.util.DistanceToShapeValueSource;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.LegacyNumericUtils;
+
+
+/**
+ * A SpatialStrategy for indexing and searching Rectangles by storing its
+ * coordinates in numeric fields. It supports all {@link SpatialOperation}s and
+ * has a custom overlap relevancy. It is based on GeoPortal's <a
+ * href="http://geoportal.svn.sourceforge.net/svnroot/geoportal/Geoportal/trunk/src/com/esri/gpt/catalog/lucene/SpatialClauseAdapter.java">SpatialClauseAdapter</a>.
+ * <p>
+ * <b>Characteristics:</b>
+ * <br>
+ * <ul>
+ * <li>Only indexes Rectangles; just one per field value. Other shapes can be provided
+ * and the bounding box will be used.</li>
+ * <li>Can query only by a Rectangle. Providing other shapes is an error.</li>
+ * <li>Supports most {@link SpatialOperation}s but not Overlaps.</li>
+ * <li>Uses the DocValues API for any sorting / relevancy.</li>
+ * </ul>
+ * <p>
+ * <b>Implementation:</b>
+ * <p>
+ * This uses 4 double fields for minX, maxX, minY, maxY
+ * and a boolean to mark a dateline cross. Depending on the particular {@link
+ * SpatialOperation}s, there are a variety of {@link org.apache.lucene.search.LegacyNumericRangeQuery}s to be
+ * done.
+ * The {@link #makeOverlapRatioValueSource(com.spatial4j.core.shape.Rectangle, double)}
+ * works by calculating the query bbox overlap percentage against the indexed
+ * shape overlap percentage. The indexed shape's coordinates are retrieved from
+ * {@link org.apache.lucene.index.LeafReader#getNumericDocValues}.
+ *
+ * @lucene.experimental
+ */
+public class BBoxStrategy extends SpatialStrategy {
+
+  public static final String SUFFIX_MINX = "__minX";
+  public static final String SUFFIX_MAXX = "__maxX";
+  public static final String SUFFIX_MINY = "__minY";
+  public static final String SUFFIX_MAXY = "__maxY";
+  public static final String SUFFIX_XDL  = "__xdl";
+
+  /*
+   * The Bounding Box gets stored as four fields for x/y min/max and a flag
+   * that says if the box crosses the dateline (xdl).
+   */
+  protected final String field_bbox;
+  protected final String field_minX;
+  protected final String field_minY;
+  protected final String field_maxX;
+  protected final String field_maxY;
+  protected final String field_xdl; // crosses dateline
+
+  protected FieldType fieldType;//for the 4 numbers
+  protected FieldType xdlFieldType;
+
+  public BBoxStrategy(SpatialContext ctx, String fieldNamePrefix) {
+    super(ctx, fieldNamePrefix);
+    field_bbox = fieldNamePrefix;
+    field_minX = fieldNamePrefix + SUFFIX_MINX;
+    field_maxX = fieldNamePrefix + SUFFIX_MAXX;
+    field_minY = fieldNamePrefix + SUFFIX_MINY;
+    field_maxY = fieldNamePrefix + SUFFIX_MAXY;
+    field_xdl = fieldNamePrefix + SUFFIX_XDL;
+
+    FieldType fieldType = new FieldType(LegacyDoubleField.TYPE_NOT_STORED);
+    fieldType.setNumericPrecisionStep(8);//Solr's default
+    fieldType.setDocValuesType(DocValuesType.NUMERIC);
+    setFieldType(fieldType);
+  }
+
+  private int getPrecisionStep() {
+    return fieldType.numericPrecisionStep();
+  }
+
+  public FieldType getFieldType() {
+    return fieldType;
+  }
+
+  /** Used to customize the indexing options of the 4 number fields, and to a lesser degree the XDL field too. Search
+   * requires indexed=true, and relevancy requires docValues. If these features aren't needed then disable them.
+   * {@link FieldType#freeze()} is called on the argument. */
+  public void setFieldType(FieldType fieldType) {
+    fieldType.freeze();
+    this.fieldType = fieldType;
+    //only double's supported right now
+    if (fieldType.numericType() != FieldType.LegacyNumericType.DOUBLE)
+      throw new IllegalArgumentException("BBoxStrategy only supports doubles at this time.");
+    //for xdlFieldType, copy some similar options. Don't do docValues since it isn't needed here.
+    xdlFieldType = new FieldType(StringField.TYPE_NOT_STORED);
+    xdlFieldType.setStored(fieldType.stored());
+    xdlFieldType.setIndexOptions(fieldType.indexOptions());
+    xdlFieldType.freeze();
+  }
+
+  //---------------------------------
+  // Indexing
+  //---------------------------------
+
+  @Override
+  public Field[] createIndexableFields(Shape shape) {
+    return createIndexableFields(shape.getBoundingBox());
+  }
+
+  public Field[] createIndexableFields(Rectangle bbox) {
+    Field[] fields = new Field[5];
+    fields[0] = new ComboField(field_minX, bbox.getMinX(), fieldType);
+    fields[1] = new ComboField(field_maxX, bbox.getMaxX(), fieldType);
+    fields[2] = new ComboField(field_minY, bbox.getMinY(), fieldType);
+    fields[3] = new ComboField(field_maxY, bbox.getMaxY(), fieldType);
+    fields[4] = new ComboField(field_xdl, bbox.getCrossesDateLine()?"T":"F", xdlFieldType);
+    return fields;
+  }
+
+  /** Field subclass circumventing Field limitations. This one instance can have any combination of indexed, stored,
+   * and docValues.
+   */
+  private static class ComboField extends Field {
+    private ComboField(String name, Object value, FieldType type) {
+      super(name, type);//this expert constructor allows us to have a field that has docValues & indexed/stored
+      super.fieldsData = value;
+    }
+
+    //Is this a hack?  We assume that numericValue() is only called for DocValues purposes.
+    @Override
+    public Number numericValue() {
+      //Numeric DocValues only supports Long,
+      final Number number = super.numericValue();
+      if (number == null)
+        return null;
+      if (fieldType().numericType() == FieldType.LegacyNumericType.DOUBLE)
+        return Double.doubleToLongBits(number.doubleValue());
+      if (fieldType().numericType() == FieldType.LegacyNumericType.FLOAT)
+        return Float.floatToIntBits(number.floatValue());
+      return number.longValue();
+    }
+  }
+
+  //---------------------------------
+  // Value Source / Relevancy
+  //---------------------------------
+
+  /**
+   * Provides access to each rectangle 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 BBoxValueSource(this);
+  }
+
+  @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 similarity based on {@link BBoxOverlapRatioValueSource}. This is just a
+   * convenience method. */
+  public ValueSource makeOverlapRatioValueSource(Rectangle queryBox, double queryTargetProportion) {
+    return new BBoxOverlapRatioValueSource(
+        makeShapeValueSource(), ctx.isGeo(), queryBox, queryTargetProportion, 0.0);
+  }
+
+  //---------------------------------
+  // Query Building
+  //---------------------------------
+
+  //  Utility on SpatialStrategy?
+//  public Query makeQueryWithValueSource(SpatialArgs args, ValueSource valueSource) {
+//    return new CustomScoreQuery(makeQuery(args), new FunctionQuery(valueSource));
+  //or...
+//  return new BooleanQuery.Builder()
+//      .add(new FunctionQuery(valueSource), BooleanClause.Occur.MUST)//matches everything and provides score
+//      .add(filterQuery, BooleanClause.Occur.FILTER)//filters (score isn't used)
+//  .build();
+//  }
+
+  @Override
+  public Query makeQuery(SpatialArgs args) {
+    Shape shape = args.getShape();
+    if (!(shape instanceof Rectangle))
+      throw new UnsupportedOperationException("Can only query by Rectangle, not " + shape);
+
+    Rectangle bbox = (Rectangle) shape;
+    Query spatial;
+
+    // Useful for understanding Relations:
+    // http://edndoc.esri.com/arcsde/9.1/general_topics/understand_spatial_relations.htm
+    SpatialOperation op = args.getOperation();
+         if( op == SpatialOperation.BBoxIntersects ) spatial = makeIntersects(bbox);
+    else if( op == SpatialOperation.BBoxWithin     ) spatial = makeWithin(bbox);
+    else if( op == SpatialOperation.Contains       ) spatial = makeContains(bbox);
+    else if( op == SpatialOperation.Intersects     ) spatial = makeIntersects(bbox);
+    else if( op == SpatialOperation.IsEqualTo      ) spatial = makeEquals(bbox);
+    else if( op == SpatialOperation.IsDisjointTo   ) spatial = makeDisjoint(bbox);
+    else if( op == SpatialOperation.IsWithin       ) spatial = makeWithin(bbox);
+    else { //no Overlaps support yet
+        throw new UnsupportedSpatialOperation(op);
+    }
+    return new ConstantScoreQuery(spatial);
+  }
+
+  /**
+   * Constructs a query to retrieve documents that fully contain the input envelope.
+   *
+   * @return the spatial query
+   */
+  Query makeContains(Rectangle bbox) {
+
+    // general case
+    // docMinX <= queryExtent.getMinX() AND docMinY <= queryExtent.getMinY() AND docMaxX >= queryExtent.getMaxX() AND docMaxY >= queryExtent.getMaxY()
+
+    // Y conditions
+    // docMinY <= queryExtent.getMinY() AND docMaxY >= queryExtent.getMaxY()
+    Query qMinY = LegacyNumericRangeQuery.newDoubleRange(field_minY, getPrecisionStep(), null, bbox.getMinY(), false, true);
+    Query qMaxY = LegacyNumericRangeQuery.newDoubleRange(field_maxY, getPrecisionStep(), bbox.getMaxY(), null, true, false);
+    Query yConditions = this.makeQuery(BooleanClause.Occur.MUST, qMinY, qMaxY);
+
+    // X conditions
+    Query xConditions;
+
+    // queries that do not cross the date line
+    if (!bbox.getCrossesDateLine()) {
+
+      // X Conditions for documents that do not cross the date line,
+      // documents that contain the min X and max X of the query envelope,
+      // docMinX <= queryExtent.getMinX() AND docMaxX >= queryExtent.getMaxX()
+      Query qMinX = LegacyNumericRangeQuery.newDoubleRange(field_minX, getPrecisionStep(), null, bbox.getMinX(), false, true);
+      Query qMaxX = LegacyNumericRangeQuery.newDoubleRange(field_maxX, getPrecisionStep(), bbox.getMaxX(), null, true, false);
+      Query qMinMax = this.makeQuery(BooleanClause.Occur.MUST, qMinX, qMaxX);
+      Query qNonXDL = this.makeXDL(false, qMinMax);
+
+      if (!ctx.isGeo()) {
+        xConditions = qNonXDL;
+      } else {
+        // X Conditions for documents that cross the date line,
+        // the left portion of the document contains the min X of the query
+        // OR the right portion of the document contains the max X of the query,
+        // docMinXLeft <= queryExtent.getMinX() OR docMaxXRight >= queryExtent.getMaxX()
+        Query qXDLLeft = LegacyNumericRangeQuery.newDoubleRange(field_minX, getPrecisionStep(), null, bbox.getMinX(), false, true);
+        Query qXDLRight = LegacyNumericRangeQuery.newDoubleRange(field_maxX, getPrecisionStep(), bbox.getMaxX(), null, true, false);
+        Query qXDLLeftRight = this.makeQuery(BooleanClause.Occur.SHOULD, qXDLLeft, qXDLRight);
+        Query qXDL = this.makeXDL(true, qXDLLeftRight);
+
+        Query qEdgeDL = null;
+        if (bbox.getMinX() == bbox.getMaxX() && Math.abs(bbox.getMinX()) == 180) {
+          double edge = bbox.getMinX() * -1;//opposite dateline edge
+          qEdgeDL = makeQuery(BooleanClause.Occur.SHOULD,
+              makeNumberTermQuery(field_minX, edge), makeNumberTermQuery(field_maxX, edge));
+        }
+
+        // apply the non-XDL and XDL conditions
+        xConditions = this.makeQuery(BooleanClause.Occur.SHOULD, qNonXDL, qXDL, qEdgeDL);
+      }
+    } else {
+      // queries that cross the date line
+
+      // No need to search for documents that do not cross the date line
+
+      // X Conditions for documents that cross the date line,
+      // the left portion of the document contains the min X of the query
+      // AND the right portion of the document contains the max X of the query,
+      // docMinXLeft <= queryExtent.getMinX() AND docMaxXRight >= queryExtent.getMaxX()
+      Query qXDLLeft = LegacyNumericRangeQuery.newDoubleRange(field_minX, getPrecisionStep(), null, bbox.getMinX(), false, true);
+      Query qXDLRight = LegacyNumericRangeQuery.newDoubleRange(field_maxX, getPrecisionStep(), bbox.getMaxX(), null, true, false);
+      Query qXDLLeftRight = this.makeXDL(true, this.makeQuery(BooleanClause.Occur.MUST, qXDLLeft, qXDLRight));
+
+      Query qWorld = makeQuery(BooleanClause.Occur.MUST,
+          makeNumberTermQuery(field_minX, -180), makeNumberTermQuery(field_maxX, 180));
+
+      xConditions = makeQuery(BooleanClause.Occur.SHOULD, qXDLLeftRight, qWorld);
+    }
+
+    // both X and Y conditions must occur
+    return this.makeQuery(BooleanClause.Occur.MUST, xConditions, yConditions);
+  }
+
+  /**
+   * Constructs a query to retrieve documents that are disjoint to the input envelope.
+   *
+   * @return the spatial query
+   */
+  Query makeDisjoint(Rectangle bbox) {
+
+    // general case
+    // docMinX > queryExtent.getMaxX() OR docMaxX < queryExtent.getMinX() OR docMinY > queryExtent.getMaxY() OR docMaxY < queryExtent.getMinY()
+
+    // Y conditions
+    // docMinY > queryExtent.getMaxY() OR docMaxY < queryExtent.getMinY()
+    Query qMinY = LegacyNumericRangeQuery.newDoubleRange(field_minY, getPrecisionStep(), bbox.getMaxY(), null, false, false);
+    Query qMaxY = LegacyNumericRangeQuery.newDoubleRange(field_maxY, getPrecisionStep(), null, bbox.getMinY(), false, false);
+    Query yConditions = this.makeQuery(BooleanClause.Occur.SHOULD, qMinY, qMaxY);
+
+    // X conditions
+    Query xConditions;
+
+    // queries that do not cross the date line
+    if (!bbox.getCrossesDateLine()) {
+
+      // X Conditions for documents that do not cross the date line,
+      // docMinX > queryExtent.getMaxX() OR docMaxX < queryExtent.getMinX()
+      Query qMinX = LegacyNumericRangeQuery.newDoubleRange(field_minX, getPrecisionStep(), bbox.getMaxX(), null, false, false);
+      if (bbox.getMinX() == -180.0 && ctx.isGeo()) {//touches dateline; -180 == 180
+        BooleanQuery.Builder bq = new BooleanQuery.Builder();
+        bq.add(qMinX, BooleanClause.Occur.MUST);
+        bq.add(makeNumberTermQuery(field_maxX, 180.0), BooleanClause.Occur.MUST_NOT);
+        qMinX = bq.build();
+      }
+      Query qMaxX = LegacyNumericRangeQuery.newDoubleRange(field_maxX, getPrecisionStep(), null, bbox.getMinX(), false, false);
+      if (bbox.getMaxX() == 180.0 && ctx.isGeo()) {//touches dateline; -180 == 180
+        BooleanQuery.Builder bq = new BooleanQuery.Builder();
+        bq.add(qMaxX, BooleanClause.Occur.MUST);
+        bq.add(makeNumberTermQuery(field_minX, -180.0), BooleanClause.Occur.MUST_NOT);
+        qMaxX = bq.build();
+      }
+      Query qMinMax = this.makeQuery(BooleanClause.Occur.SHOULD, qMinX, qMaxX);
+      Query qNonXDL = this.makeXDL(false, qMinMax);
+
+      if (!ctx.isGeo()) {
+        xConditions = qNonXDL;
+      } else {
+        // X Conditions for documents that cross the date line,
+
+        // both the left and right portions of the document must be disjoint to the query
+        // (docMinXLeft > queryExtent.getMaxX() OR docMaxXLeft < queryExtent.getMinX()) AND
+        // (docMinXRight > queryExtent.getMaxX() OR docMaxXRight < queryExtent.getMinX())
+        // where: docMaxXLeft = 180.0, docMinXRight = -180.0
+        // (docMaxXLeft  < queryExtent.getMinX()) equates to (180.0  < queryExtent.getMinX()) and is ignored
+        // (docMinXRight > queryExtent.getMaxX()) equates to (-180.0 > queryExtent.getMaxX()) and is ignored
+        Query qMinXLeft = LegacyNumericRangeQuery.newDoubleRange(field_minX, getPrecisionStep(), bbox.getMaxX(), null, false, false);
+        Query qMaxXRight = LegacyNumericRangeQuery.newDoubleRange(field_maxX, getPrecisionStep(), null, bbox.getMinX(), false, false);
+        Query qLeftRight = this.makeQuery(BooleanClause.Occur.MUST, qMinXLeft, qMaxXRight);
+        Query qXDL = this.makeXDL(true, qLeftRight);
+
+        // apply the non-XDL and XDL conditions
+        xConditions = this.makeQuery(BooleanClause.Occur.SHOULD, qNonXDL, qXDL);
+      }
+      // queries that cross the date line
+    } else {
+
+      // X Conditions for documents that do not cross the date line,
+      // the document must be disjoint to both the left and right query portions
+      // (docMinX > queryExtent.getMaxX()Left OR docMaxX < queryExtent.getMinX()) AND (docMinX > queryExtent.getMaxX() OR docMaxX < queryExtent.getMinX()Left)
+      // where: queryExtent.getMaxX()Left = 180.0, queryExtent.getMinX()Left = -180.0
+      Query qMinXLeft = LegacyNumericRangeQuery.newDoubleRange(field_minX, getPrecisionStep(), 180.0, null, false, false);
+      Query qMaxXLeft = LegacyNumericRangeQuery.newDoubleRange(field_maxX, getPrecisionStep(), null, bbox.getMinX(), false, false);
+      Query qMinXRight = LegacyNumericRangeQuery.newDoubleRange(field_minX, getPrecisionStep(), bbox.getMaxX(), null, false, false);
+      Query qMaxXRight = LegacyNumericRangeQuery.newDoubleRange(field_maxX, getPrecisionStep(), null, -180.0, false, false);
+      Query qLeft = this.makeQuery(BooleanClause.Occur.SHOULD, qMinXLeft, qMaxXLeft);
+      Query qRight = this.makeQuery(BooleanClause.Occur.SHOULD, qMinXRight, qMaxXRight);
+      Query qLeftRight = this.makeQuery(BooleanClause.Occur.MUST, qLeft, qRight);
+
+      // No need to search for documents that do not cross the date line
+
+      xConditions = this.makeXDL(false, qLeftRight);
+    }
+
+    // either X or Y conditions should occur
+    return this.makeQuery(BooleanClause.Occur.SHOULD, xConditions, yConditions);
+  }
+
+  /**
+   * Constructs a query to retrieve documents that equal the input envelope.
+   *
+   * @return the spatial query
+   */
+  Query makeEquals(Rectangle bbox) {
+
+    // docMinX = queryExtent.getMinX() AND docMinY = queryExtent.getMinY() AND docMaxX = queryExtent.getMaxX() AND docMaxY = queryExtent.getMaxY()
+    Query qMinX = makeNumberTermQuery(field_minX, bbox.getMinX());
+    Query qMinY = makeNumberTermQuery(field_minY, bbox.getMinY());
+    Query qMaxX = makeNumberTermQuery(field_maxX, bbox.getMaxX());
+    Query qMaxY = makeNumberTermQuery(field_maxY, bbox.getMaxY());
+    return makeQuery(BooleanClause.Occur.MUST, qMinX, qMinY, qMaxX, qMaxY);
+  }
+
+  /**
+   * Constructs a query to retrieve documents that intersect the input envelope.
+   *
+   * @return the spatial query
+   */
+  Query makeIntersects(Rectangle bbox) {
+
+    // the original intersects query does not work for envelopes that cross the date line,
+    // switch to a NOT Disjoint query
+
+    // MUST_NOT causes a problem when it's the only clause type within a BooleanQuery,
+    // to get around it we add all documents as a SHOULD
+
+    // there must be an envelope, it must not be disjoint
+    Query qHasEnv;
+    if (ctx.isGeo()) {
+      Query qIsNonXDL = this.makeXDL(false);
+      Query qIsXDL = ctx.isGeo() ? this.makeXDL(true) : null;
+      qHasEnv = this.makeQuery(BooleanClause.Occur.SHOULD, qIsNonXDL, qIsXDL);
+    } else {
+      qHasEnv = this.makeXDL(false);
+    }
+
+    BooleanQuery.Builder qNotDisjoint = new BooleanQuery.Builder();
+    qNotDisjoint.add(qHasEnv, BooleanClause.Occur.MUST);
+    Query qDisjoint = makeDisjoint(bbox);
+    qNotDisjoint.add(qDisjoint, BooleanClause.Occur.MUST_NOT);
+
+    //Query qDisjoint = makeDisjoint();
+    //BooleanQuery qNotDisjoint = new BooleanQuery();
+    //qNotDisjoint.add(new MatchAllDocsQuery(),BooleanClause.Occur.SHOULD);
+    //qNotDisjoint.add(qDisjoint,BooleanClause.Occur.MUST_NOT);
+    return qNotDisjoint.build();
+  }
+
+  /**
+   * Makes a boolean query based upon a collection of queries and a logical operator.
+   *
+   * @param occur the logical operator
+   * @param queries the query collection
+   * @return the query
+   */
+  BooleanQuery makeQuery(BooleanClause.Occur occur, Query... queries) {
+    BooleanQuery.Builder bq = new BooleanQuery.Builder();
+    for (Query query : queries) {
+      if (query != null)
+        bq.add(query, occur);
+    }
+    return bq.build();
+  }
+
+  /**
+   * Constructs a query to retrieve documents are fully within the input envelope.
+   *
+   * @return the spatial query
+   */
+  Query makeWithin(Rectangle bbox) {
+
+    // general case
+    // docMinX >= queryExtent.getMinX() AND docMinY >= queryExtent.getMinY() AND docMaxX <= queryExtent.getMaxX() AND docMaxY <= queryExtent.getMaxY()
+
+    // Y conditions
+    // docMinY >= queryExtent.getMinY() AND docMaxY <= queryExtent.getMaxY()
+    Query qMinY = LegacyNumericRangeQuery.newDoubleRange(field_minY, getPrecisionStep(), bbox.getMinY(), null, true, false);
+    Query qMaxY = LegacyNumericRangeQuery.newDoubleRange(field_maxY, getPrecisionStep(), null, bbox.getMaxY(), false, true);
+    Query yConditions = this.makeQuery(BooleanClause.Occur.MUST, qMinY, qMaxY);
+
+    // X conditions
+    Query xConditions;
+
+    if (ctx.isGeo() && bbox.getMinX() == -180.0 && bbox.getMaxX() == 180.0) {
+      //if query world-wraps, only the y condition matters
+      return yConditions;
+
+    } else if (!bbox.getCrossesDateLine()) {
+      // queries that do not cross the date line
+
+      // docMinX >= queryExtent.getMinX() AND docMaxX <= queryExtent.getMaxX()
+      Query qMinX = LegacyNumericRangeQuery.newDoubleRange(field_minX, getPrecisionStep(), bbox.getMinX(), null, true, false);
+      Query qMaxX = LegacyNumericRangeQuery.newDoubleRange(field_maxX, getPrecisionStep(), null, bbox.getMaxX(), false, true);
+      Query qMinMax = this.makeQuery(BooleanClause.Occur.MUST, qMinX, qMaxX);
+
+      double edge = 0;//none, otherwise opposite dateline of query
+      if (bbox.getMinX() == -180.0)
+        edge = 180;
+      else if (bbox.getMaxX() == 180.0)
+        edge = -180;
+      if (edge != 0 && ctx.isGeo()) {
+        Query edgeQ = makeQuery(BooleanClause.Occur.MUST,
+            makeNumberTermQuery(field_minX, edge), makeNumberTermQuery(field_maxX, edge));
+        qMinMax = makeQuery(BooleanClause.Occur.SHOULD, qMinMax, edgeQ);
+      }
+
+      xConditions = this.makeXDL(false, qMinMax);
+
+      // queries that cross the date line
+    } else {
+
+      // X Conditions for documents that do not cross the date line
+
+      // the document should be within the left portion of the query
+      // docMinX >= queryExtent.getMinX() AND docMaxX <= 180.0
+      Query qMinXLeft = LegacyNumericRangeQuery.newDoubleRange(field_minX, getPrecisionStep(), bbox.getMinX(), null, true, false);
+      Query qMaxXLeft = LegacyNumericRangeQuery.newDoubleRange(field_maxX, getPrecisionStep(), null, 180.0, false, true);
+      Query qLeft = this.makeQuery(BooleanClause.Occur.MUST, qMinXLeft, qMaxXLeft);
+
+      // the document should be within the right portion of the query
+      // docMinX >= -180.0 AND docMaxX <= queryExtent.getMaxX()
+      Query qMinXRight = LegacyNumericRangeQuery.newDoubleRange(field_minX, getPrecisionStep(), -180.0, null, true, false);
+      Query qMaxXRight = LegacyNumericRangeQuery.newDoubleRange(field_maxX, getPrecisionStep(), null, bbox.getMaxX(), false, true);
+      Query qRight = this.makeQuery(BooleanClause.Occur.MUST, qMinXRight, qMaxXRight);
+
+      // either left or right conditions should occur,
+      // apply the left and right conditions to documents that do not cross the date line
+      Query qLeftRight = this.makeQuery(BooleanClause.Occur.SHOULD, qLeft, qRight);
+      Query qNonXDL = this.makeXDL(false, qLeftRight);
+
+      // X Conditions for documents that cross the date line,
+      // the left portion of the document must be within the left portion of the query,
+      // AND the right portion of the document must be within the right portion of the query
+      // docMinXLeft >= queryExtent.getMinX() AND docMaxXLeft <= 180.0
+      // AND docMinXRight >= -180.0 AND docMaxXRight <= queryExtent.getMaxX()
+      Query qXDLLeft = LegacyNumericRangeQuery.newDoubleRange(field_minX, getPrecisionStep(), bbox.getMinX(), null, true, false);
+      Query qXDLRight = LegacyNumericRangeQuery.newDoubleRange(field_maxX, getPrecisionStep(), null, bbox.getMaxX(), false, true);
+      Query qXDLLeftRight = this.makeQuery(BooleanClause.Occur.MUST, qXDLLeft, qXDLRight);
+      Query qXDL = this.makeXDL(true, qXDLLeftRight);
+
+      // apply the non-XDL and XDL conditions
+      xConditions = this.makeQuery(BooleanClause.Occur.SHOULD, qNonXDL, qXDL);
+    }
+
+    // both X and Y conditions must occur
+    return this.makeQuery(BooleanClause.Occur.MUST, xConditions, yConditions);
+  }
+
+  /**
+   * Constructs a query to retrieve documents that do or do not cross the date line.
+   *
+   * @param crossedDateLine <code>true</true> for documents that cross the date line
+   * @return the query
+   */
+  private Query makeXDL(boolean crossedDateLine) {
+    // The 'T' and 'F' values match solr fields
+    return new TermQuery(new Term(field_xdl, crossedDateLine ? "T" : "F"));
+  }
+
+  /**
+   * Constructs a query to retrieve documents that do or do not cross the date line
+   * and match the supplied spatial query.
+   *
+   * @param crossedDateLine <code>true</true> for documents that cross the date line
+   * @param query the spatial query
+   * @return the query
+   */
+  private Query makeXDL(boolean crossedDateLine, Query query) {
+    if (!ctx.isGeo()) {
+      assert !crossedDateLine;
+      return query;
+    }
+    BooleanQuery.Builder bq = new BooleanQuery.Builder();
+    bq.add(this.makeXDL(crossedDateLine), BooleanClause.Occur.MUST);
+    bq.add(query, BooleanClause.Occur.MUST);
+    return bq.build();
+  }
+
+  private Query makeNumberTermQuery(String field, double number) {
+    BytesRefBuilder bytes = new BytesRefBuilder();
+    LegacyNumericUtils.longToPrefixCoded(LegacyNumericUtils.doubleToSortableLong(number), 0, bytes);
+    return new TermQuery(new Term(field, bytes.get()));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxValueSource.java
new file mode 100644
index 0000000..5d95407
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxValueSource.java
@@ -0,0 +1,115 @@
+/*
+ * 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.bbox;
+
+import com.spatial4j.core.shape.Rectangle;
+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.search.Explanation;
+import org.apache.lucene.util.Bits;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * A ValueSource in which the indexed Rectangle is returned from
+ * {@link org.apache.lucene.queries.function.FunctionValues#objectVal(int)}.
+ *
+ * @lucene.internal
+ */
+class BBoxValueSource extends ValueSource {
+
+  private final BBoxStrategy strategy;
+
+  public BBoxValueSource(BBoxStrategy strategy) {
+    this.strategy = strategy;
+  }
+
+  @Override
+  public String description() {
+    return "bboxShape(" + strategy.getFieldName() + ")";
+  }
+
+  @Override
+  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
+    LeafReader reader = readerContext.reader();
+    final NumericDocValues minX = DocValues.getNumeric(reader, strategy.field_minX);
+    final NumericDocValues minY = DocValues.getNumeric(reader, strategy.field_minY);
+    final NumericDocValues maxX = DocValues.getNumeric(reader, strategy.field_maxX);
+    final NumericDocValues maxY = DocValues.getNumeric(reader, strategy.field_maxY);
+
+    final Bits validBits = DocValues.getDocsWithField(reader, strategy.field_minX);//could have chosen any field
+    //reused
+    final Rectangle rect = strategy.getSpatialContext().makeRectangle(0,0,0,0);
+
+    return new FunctionValues() {
+      @Override
+      public Object objectVal(int doc) {
+        if (!validBits.get(doc)) {
+          return null;
+        } else {
+          rect.reset(
+              Double.longBitsToDouble(minX.get(doc)), Double.longBitsToDouble(maxX.get(doc)),
+              Double.longBitsToDouble(minY.get(doc)), Double.longBitsToDouble(maxY.get(doc)));
+          return rect;
+        }
+      }
+
+      @Override
+      public String strVal(int doc) {//TODO support WKT output once Spatial4j does
+        Object v = objectVal(doc);
+        return v == null ? null : v.toString();
+      }
+
+      @Override
+      public boolean exists(int doc) {
+        return validBits.get(doc);
+      }
+
+      @Override
+      public Explanation explain(int doc) {
+        return Explanation.match(Float.NaN, toString(doc));
+      }
+
+      @Override
+      public String toString(int doc) {
+        return description() + '=' + strVal(doc);
+      }
+    };
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    BBoxValueSource that = (BBoxValueSource) o;
+
+    if (!strategy.equals(that.strategy)) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    return strategy.hashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/package-info.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/package-info.java
new file mode 100644
index 0000000..518f447
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * Bounding Box Spatial Strategy
+ * <p>
+ * Index a shape extent using 4 numeric fields and a flag to say if it crosses the dateline
+ */
+package org.apache.lucene.spatial.bbox;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeSpatialStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeSpatialStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeSpatialStrategy.java
new file mode 100644
index 0000000..7dc2dfa
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeSpatialStrategy.java
@@ -0,0 +1,144 @@
+/*
+ * 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.composite;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import com.spatial4j.core.shape.Point;
+import com.spatial4j.core.shape.Shape;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.spatial.SpatialStrategy;
+import org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy;
+import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
+import org.apache.lucene.spatial.query.SpatialArgs;
+import org.apache.lucene.spatial.query.SpatialOperation;
+import org.apache.lucene.spatial.query.UnsupportedSpatialOperation;
+import org.apache.lucene.spatial.serialized.SerializedDVStrategy;
+import org.apache.lucene.spatial.util.ShapePredicateValueSource;
+
+/**
+ * A composite {@link SpatialStrategy} based on {@link RecursivePrefixTreeStrategy} (RPT) and
+ * {@link SerializedDVStrategy} (SDV).
+ * RPT acts as an index to the precision available in SDV, and in some circumstances can avoid geometry lookups based
+ * on where a cell is in relation to the query shape.  Currently the only predicate optimized like this is Intersects.
+ * All predicates are supported except for the BBox* ones, and Disjoint.
+ *
+ * @lucene.experimental
+ */
+public class CompositeSpatialStrategy extends SpatialStrategy {
+
+  //TODO support others? (BBox)
+  private final RecursivePrefixTreeStrategy indexStrategy;
+
+  /** Has the geometry. */ // TODO support others?
+  private final SerializedDVStrategy geometryStrategy;
+  private boolean optimizePredicates = true;
+
+  public CompositeSpatialStrategy(String fieldName,
+                                  RecursivePrefixTreeStrategy indexStrategy, SerializedDVStrategy geometryStrategy) {
+    super(indexStrategy.getSpatialContext(), fieldName);//field name; unused
+    this.indexStrategy = indexStrategy;
+    this.geometryStrategy = geometryStrategy;
+  }
+
+  public RecursivePrefixTreeStrategy getIndexStrategy() {
+    return indexStrategy;
+  }
+
+  public SerializedDVStrategy getGeometryStrategy() {
+    return geometryStrategy;
+  }
+
+  public boolean isOptimizePredicates() {
+    return optimizePredicates;
+  }
+
+  /** Set to false to NOT use optimized search predicates that avoid checking the geometry sometimes. Only useful for
+   * benchmarking. */
+  public void setOptimizePredicates(boolean optimizePredicates) {
+    this.optimizePredicates = optimizePredicates;
+  }
+
+  @Override
+  public Field[] createIndexableFields(Shape shape) {
+    List<Field> fields = new ArrayList<>();
+    Collections.addAll(fields, indexStrategy.createIndexableFields(shape));
+    Collections.addAll(fields, geometryStrategy.createIndexableFields(shape));
+    return fields.toArray(new Field[fields.size()]);
+  }
+
+  @Override
+  public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
+    //TODO consider indexing center-point in DV?  Guarantee contained by the shape, which could then be used for
+    // other purposes like faster WITHIN predicate?
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Query makeQuery(SpatialArgs args) {
+    final SpatialOperation pred = args.getOperation();
+
+    if (pred == SpatialOperation.BBoxIntersects || pred == SpatialOperation.BBoxWithin) {
+      throw new UnsupportedSpatialOperation(pred);
+    }
+
+    if (pred == SpatialOperation.IsDisjointTo) {
+//      final Query intersectQuery = makeQuery(new SpatialArgs(SpatialOperation.Intersects, args.getShape()));
+//      DocValues.getDocsWithField(reader, geometryStrategy.getFieldName());
+      //TODO resurrect Disjoint spatial query utility accepting a field name known to have DocValues.
+      // update class docs when it's added.
+      throw new UnsupportedSpatialOperation(pred);
+    }
+
+    final ShapePredicateValueSource predicateValueSource =
+        new ShapePredicateValueSource(geometryStrategy.makeShapeValueSource(), pred, args.getShape());
+    //System.out.println("PredOpt: " + optimizePredicates);
+    if (pred == SpatialOperation.Intersects && optimizePredicates) {
+      // We have a smart Intersects impl
+
+      final SpatialPrefixTree grid = indexStrategy.getGrid();
+      final int detailLevel = grid.getLevelForDistance(args.resolveDistErr(ctx, 0.0));//default to max precision
+      return new IntersectsRPTVerifyQuery(args.getShape(), indexStrategy.getFieldName(), grid,
+          detailLevel, indexStrategy.getPrefixGridScanLevel(), predicateValueSource);
+    } else {
+      //The general path; all index matches get verified
+
+      SpatialArgs indexArgs;
+      if (pred == SpatialOperation.Contains) {
+        // note: we could map IsWithin as well but it's pretty darned slow since it touches all world grids
+        indexArgs = args;
+      } else {
+        //TODO add args.clone method with new predicate? Or simply make non-final?
+        indexArgs = new SpatialArgs(SpatialOperation.Intersects, args.getShape());
+        indexArgs.setDistErr(args.getDistErr());
+        indexArgs.setDistErrPct(args.getDistErrPct());
+      }
+
+      if (indexArgs.getDistErr() == null && indexArgs.getDistErrPct() == null) {
+        indexArgs.setDistErrPct(0.10);
+      }
+
+      final Query indexQuery = indexStrategy.makeQuery(indexArgs);
+      return new CompositeVerifyQuery(indexQuery, predicateValueSource);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeVerifyQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeVerifyQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeVerifyQuery.java
new file mode 100644
index 0000000..0e6ea2c
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeVerifyQuery.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.composite;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.lucene.index.IndexReader;
+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.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TwoPhaseIterator;
+import org.apache.lucene.search.Weight;
+
+/**
+ * A Query that considers an "indexQuery" to have approximate results, and a follow-on
+ * {@link ValueSource}/{@link FunctionValues#boolVal(int)} is called to verify each hit
+ * from {@link TwoPhaseIterator#matches()}.
+ *
+ * @lucene.experimental
+ */
+public class CompositeVerifyQuery extends Query {
+  final Query indexQuery;//approximation (matches more than needed)
+  final ValueSource predicateValueSource;//we call boolVal(doc)
+
+  public CompositeVerifyQuery(Query indexQuery, ValueSource predicateValueSource) {
+    this.indexQuery = indexQuery;
+    this.predicateValueSource = predicateValueSource;
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    final Query rewritten = indexQuery.rewrite(reader);
+    if (rewritten != indexQuery) {
+      return new CompositeVerifyQuery(rewritten, predicateValueSource);
+    }
+    return super.rewrite(reader);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    if (!super.equals(o)) return false;
+
+    CompositeVerifyQuery that = (CompositeVerifyQuery) o;
+
+    if (!indexQuery.equals(that.indexQuery)) return false;
+    if (!predicateValueSource.equals(that.predicateValueSource)) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    result = 31 * result + indexQuery.hashCode();
+    result = 31 * result + predicateValueSource.hashCode();
+    return result;
+  }
+
+  @Override
+  public String toString(String field) {
+    //TODO verify this looks good
+    return getClass().getSimpleName() + "(" + indexQuery.toString(field) + ", " + predicateValueSource + ")";
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+    final Weight indexQueryWeight = indexQuery.createWeight(searcher, false);//scores aren't unsupported
+    final Map valueSourceContext = ValueSource.newContext(searcher);
+
+    return new ConstantScoreWeight(this) {
+
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+
+        final Scorer indexQueryScorer = indexQueryWeight.scorer(context);
+        if (indexQueryScorer == null) {
+          return null;
+        }
+
+        final FunctionValues predFuncValues = predicateValueSource.getValues(valueSourceContext, context);
+
+        final TwoPhaseIterator twoPhaseIterator = new TwoPhaseIterator(indexQueryScorer.iterator()) {
+          @Override
+          public boolean matches() throws IOException {
+            return predFuncValues.boolVal(indexQueryScorer.docID());
+          }
+
+          @Override
+          public float matchCost() {
+            return 100; // TODO: use cost of predFuncValues.boolVal()
+          }
+        };
+
+        return new ConstantScoreScorer(this, score(), twoPhaseIterator);
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java
new file mode 100644
index 0000000..a963b6e
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java
@@ -0,0 +1,235 @@
+/*
+ * 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.composite;
+
+import java.io.IOException;
+import java.util.Map;
+
+import com.spatial4j.core.shape.Shape;
+import com.spatial4j.core.shape.SpatialRelation;
+
+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.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TwoPhaseIterator;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.spatial.prefix.AbstractVisitingPrefixTreeQuery;
+import org.apache.lucene.spatial.prefix.tree.Cell;
+import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+/**
+ * A spatial Intersects predicate that distinguishes an approximated match from an exact match based on which cells
+ * are within the query shape. It exposes a {@link TwoPhaseIterator} that will verify a match with a provided
+ * predicate in the form of a {@link ValueSource} by calling {@link FunctionValues#boolVal(int)}.
+ *
+ * @lucene.internal
+ */
+public class IntersectsRPTVerifyQuery extends Query {
+
+  private final IntersectsDifferentiatingQuery intersectsDiffQuery;
+  private final ValueSource predicateValueSource; // we call FunctionValues.boolVal(doc)
+
+  public IntersectsRPTVerifyQuery(Shape queryShape, String fieldName, SpatialPrefixTree grid, int detailLevel,
+                                  int prefixGridScanLevel, ValueSource predicateValueSource) {
+    this.predicateValueSource = predicateValueSource;
+    this.intersectsDiffQuery = new IntersectsDifferentiatingQuery(queryShape, fieldName, grid, detailLevel,
+        prefixGridScanLevel);
+  }
+
+  @Override
+  public String toString(String field) {
+    return "IntersectsVerified(fieldName=" + field + ")";
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (!super.equals(o)) return false;
+
+    IntersectsRPTVerifyQuery that = (IntersectsRPTVerifyQuery) o;
+
+    if (!intersectsDiffQuery.equals(that.intersectsDiffQuery)) return false;
+    return predicateValueSource.equals(that.predicateValueSource);
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    result = 31 * result + intersectsDiffQuery.hashCode();
+    result = 31 * result + predicateValueSource.hashCode();
+    return result;
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+    final Map valueSourceContext = ValueSource.newContext(searcher);
+
+    return new ConstantScoreWeight(this) {
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        // Compute approx & exact
+        final IntersectsDifferentiatingQuery.IntersectsDifferentiatingVisitor result =
+            intersectsDiffQuery.compute(context);
+        if (result.approxDocIdSet == null) {
+          return null;
+        }
+        final DocIdSetIterator approxDISI = result.approxDocIdSet.iterator();
+        if (approxDISI == null) {
+          return null;
+        }
+        final DocIdSetIterator exactIterator;
+        if (result.exactDocIdSet != null) {
+          // If both sets are the same, there's nothing to verify; we needn't return a TwoPhaseIterator
+          if (result.approxDocIdSet == result.exactDocIdSet) {
+            return new ConstantScoreScorer(this, score(), approxDISI);
+          }
+          exactIterator = result.exactDocIdSet.iterator();
+          assert exactIterator != null;
+        } else {
+          exactIterator = null;
+        }
+
+        final FunctionValues predFuncValues = predicateValueSource.getValues(valueSourceContext, context);
+
+        final TwoPhaseIterator twoPhaseIterator = new TwoPhaseIterator(approxDISI) {
+          @Override
+          public boolean matches() throws IOException {
+            final int doc = approxDISI.docID();
+            if (exactIterator != null) {
+              if (exactIterator.docID() < doc) {
+                exactIterator.advance(doc);
+              }
+              if (exactIterator.docID() == doc) {
+                return true;
+              }
+            }
+
+            return predFuncValues.boolVal(doc);
+          }
+
+          @Override
+          public float matchCost() {
+            return 100; // TODO: use cost of exactIterator.advance() and predFuncValues.boolVal()
+          }
+        };
+
+        return new ConstantScoreScorer(this, score(), twoPhaseIterator);
+      }
+    };
+  }
+
+  //This may be a "Query" but we don't use it as-such; the caller calls the constructor and then compute() and examines
+  // the results which consists of two parts -- the approximated results, and a subset of exact matches. The
+  // difference needs to be verified.
+  // TODO refactor AVPTQ to not be a Query?
+  private static class IntersectsDifferentiatingQuery extends AbstractVisitingPrefixTreeQuery {
+
+    public IntersectsDifferentiatingQuery(Shape queryShape, String fieldName, SpatialPrefixTree grid,
+                                          int detailLevel, int prefixGridScanLevel) {
+      super(queryShape, fieldName, grid, detailLevel, prefixGridScanLevel);
+    }
+
+    IntersectsDifferentiatingQuery.IntersectsDifferentiatingVisitor compute(LeafReaderContext context)
+        throws IOException {
+      final IntersectsDifferentiatingQuery.IntersectsDifferentiatingVisitor result =
+          new IntersectsDifferentiatingQuery.IntersectsDifferentiatingVisitor(context);
+      result.getDocIdSet();//computes
+      return result;
+    }
+
+    // TODO consider if IntersectsPrefixTreeQuery should simply do this and provide both sets
+
+    class IntersectsDifferentiatingVisitor extends VisitorTemplate {
+      DocIdSetBuilder approxBuilder = new DocIdSetBuilder(maxDoc);
+      DocIdSetBuilder exactBuilder = new DocIdSetBuilder(maxDoc);
+      boolean approxIsEmpty = true;
+      boolean exactIsEmpty = true;
+      DocIdSet exactDocIdSet;
+      DocIdSet approxDocIdSet;
+
+      public IntersectsDifferentiatingVisitor(LeafReaderContext context) throws IOException {
+        super(context);
+      }
+
+      @Override
+      protected void start() throws IOException {
+      }
+
+      @Override
+      protected DocIdSet finish() throws IOException {
+        if (exactIsEmpty) {
+          exactDocIdSet = null;
+        } else {
+          exactDocIdSet = exactBuilder.build();
+        }
+        if (approxIsEmpty) {
+          approxDocIdSet = exactDocIdSet;//optimization
+        } else {
+          if (exactDocIdSet != null) {
+            approxBuilder.add(exactDocIdSet.iterator());
+          }
+          approxDocIdSet = approxBuilder.build();
+        }
+        return null;//unused in this weird re-use of AVPTQ
+      }
+
+      @Override
+      protected boolean visitPrefix(Cell cell) throws IOException {
+        if (cell.getShapeRel() == SpatialRelation.WITHIN) {
+          exactIsEmpty = false;
+          collectDocs(exactBuilder);//note: we'll add exact to approx on finish()
+          return false;
+        } else if (cell.getLevel() == detailLevel) {
+          approxIsEmpty = false;
+          collectDocs(approxBuilder);
+          return false;
+        }
+        return true;
+      }
+
+      @Override
+      protected void visitLeaf(Cell cell) throws IOException {
+        if (cell.getShapeRel() == SpatialRelation.WITHIN) {
+          exactIsEmpty = false;
+          collectDocs(exactBuilder);//note: we'll add exact to approx on finish()
+        } else {
+          approxIsEmpty = false;
+          collectDocs(approxBuilder);
+        }
+      }
+    }
+
+    @Override
+    public DocIdSet getDocIdSet(LeafReaderContext context) throws IOException {
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public String toString(String field) {
+      throw new IllegalStateException();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/package-info.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/package-info.java
new file mode 100644
index 0000000..c207ea6
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/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.
+ */
+
+/** Composite strategies. */
+package org.apache.lucene.spatial.composite;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/package-info.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/package-info.java
new file mode 100644
index 0000000..c86bc6e
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/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.
+ */
+
+/**
+ * Lucene advanced spatial search
+ */
+package org.apache.lucene.spatial;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeQuery.java
new file mode 100644
index 0000000..127e689
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeQuery.java
@@ -0,0 +1,133 @@
+/*
+ * 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;
+
+import java.io.IOException;
+
+import com.spatial4j.core.shape.Shape;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+/**
+ * Base class for Lucene Queries on SpatialPrefixTree fields.
+ * @lucene.internal
+ */
+public abstract class AbstractPrefixTreeQuery extends Query {
+
+  protected final Shape queryShape;
+  protected final String fieldName;
+  protected final SpatialPrefixTree grid;//not in equals/hashCode since it's implied for a specific field
+  protected final int detailLevel;
+
+  public AbstractPrefixTreeQuery(Shape queryShape, String fieldName, SpatialPrefixTree grid, int detailLevel) {
+    this.queryShape = queryShape;
+    this.fieldName = fieldName;
+    this.grid = grid;
+    this.detailLevel = detailLevel;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (super.equals(o) == false) return false;
+
+    AbstractPrefixTreeQuery that = (AbstractPrefixTreeQuery) o;
+
+    if (detailLevel != that.detailLevel) return false;
+    if (!fieldName.equals(that.fieldName)) return false;
+    if (!queryShape.equals(that.queryShape)) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    result = 31 * result + queryShape.hashCode();
+    result = 31 * result + fieldName.hashCode();
+    result = 31 * result + detailLevel;
+    return result;
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+    return new ConstantScoreWeight(this) {
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        DocIdSet docSet = getDocIdSet(context);
+        if (docSet == null) {
+          return null;
+        }
+        DocIdSetIterator disi = docSet.iterator();
+        if (disi == null) {
+          return null;
+        }
+        return new ConstantScoreScorer(this, score(), disi);
+      }
+    };
+  }
+
+  protected abstract DocIdSet getDocIdSet(LeafReaderContext context) throws IOException;
+
+  /** Holds transient state and docid collecting utility methods as part of
+   * traversing a {@link TermsEnum} for a {@link org.apache.lucene.index.LeafReaderContext}. */
+  public abstract class BaseTermsEnumTraverser {//TODO rename to LeafTermsEnumTraverser ?
+    //note: only 'fieldName' (accessed in constructor) keeps this from being a static inner class
+
+    protected final LeafReaderContext context;
+    protected final int maxDoc;
+
+    protected TermsEnum termsEnum;//remember to check for null!
+    protected PostingsEnum postingsEnum;
+
+    public BaseTermsEnumTraverser(LeafReaderContext context) throws IOException {
+      this.context = context;
+      LeafReader reader = context.reader();
+      this.maxDoc = reader.maxDoc();
+      Terms terms = reader.terms(fieldName);
+      if (terms != null)
+        this.termsEnum = terms.iterator();
+    }
+
+    protected void collectDocs(BitSet bitSet) throws IOException {
+      assert termsEnum != null;
+      postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
+      bitSet.or(postingsEnum);
+    }
+
+    protected void collectDocs(DocIdSetBuilder docSetBuilder) throws IOException {
+      assert termsEnum != null;
+      postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
+      docSetBuilder.add(postingsEnum);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeQuery.java
new file mode 100644
index 0000000..2237ca9
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeQuery.java
@@ -0,0 +1,380 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import com.spatial4j.core.shape.Shape;
+import com.spatial4j.core.shape.SpatialRelation;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.spatial.prefix.tree.Cell;
+import org.apache.lucene.spatial.prefix.tree.CellIterator;
+import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Traverses a {@link SpatialPrefixTree} indexed field, using the template and
+ * visitor design patterns for subclasses to guide the traversal and collect
+ * matching documents.
+ * <p>
+ * Subclasses implement {@link #getDocIdSet(org.apache.lucene.index.LeafReaderContext)}
+ * by instantiating a custom {@link VisitorTemplate} subclass (i.e. an anonymous inner class)
+ * and implement the required methods.
+ *
+ * @lucene.internal
+ */
+public abstract class AbstractVisitingPrefixTreeQuery extends AbstractPrefixTreeQuery {
+
+  //Historical note: this code resulted from a refactoring of RecursivePrefixTreeQuery,
+  // which in turn came out of SOLR-2155
+
+  //This class perhaps could have been implemented in terms of FilteredTermsEnum & MultiTermQuery.
+  //  Maybe so for simple Intersects predicate but not for when we want to collect terms
+  //  differently depending on cell state like IsWithin and for fuzzy/accurate collection planned improvements.  At
+  //  least it would just make things more complicated.
+
+  protected final int prefixGridScanLevel;//at least one less than grid.getMaxLevels()
+
+  public AbstractVisitingPrefixTreeQuery(Shape queryShape, String fieldName, SpatialPrefixTree grid,
+                                         int detailLevel, int prefixGridScanLevel) {
+    super(queryShape, fieldName, grid, detailLevel);
+    this.prefixGridScanLevel = Math.max(0, Math.min(prefixGridScanLevel, grid.getMaxLevels() - 1));
+    assert detailLevel <= grid.getMaxLevels();
+  }
+
+  /**
+   * An abstract class designed to make it easy to implement predicates or
+   * other operations on a {@link SpatialPrefixTree} indexed field. An instance
+   * of this class is not designed to be re-used across LeafReaderContext
+   * instances so simply create a new one per-leaf.
+   * The {@link #getDocIdSet()} method here starts the work. It first checks
+   * that there are indexed terms; if not it quickly returns null. Then it calls
+   * {@link #start()} so a subclass can set up a return value, like an
+   * {@link org.apache.lucene.util.FixedBitSet}. Then it starts the traversal
+   * process, calling {@link #findSubCellsToVisit(org.apache.lucene.spatial.prefix.tree.Cell)}
+   * which by default finds the top cells that intersect {@code queryShape}. If
+   * there isn't an indexed cell for a corresponding cell returned for this
+   * method then it's short-circuited until it finds one, at which point
+   * {@link #visitPrefix(org.apache.lucene.spatial.prefix.tree.Cell)} is called. At
+   * some depths, of the tree, the algorithm switches to a scanning mode that
+   * calls {@link #visitScanned(org.apache.lucene.spatial.prefix.tree.Cell)}
+   * for each leaf cell found.
+   *
+   * @lucene.internal
+   */
+  public abstract class VisitorTemplate extends BaseTermsEnumTraverser {
+
+  /* Future potential optimizations:
+
+  * Can a polygon query shape be optimized / made-simpler at recursive depths
+    (e.g. intersection of shape + cell box)
+
+  * RE "scan" vs divide & conquer performance decision:
+    We should use termsEnum.docFreq() as an estimate on the number of places at
+    this depth.  It would be nice if termsEnum knew how many terms
+    start with the current term without having to repeatedly next() & test to find out.
+
+  * Perhaps don't do intermediate seek()'s to cells above detailLevel that have Intersects
+    relation because we won't be collecting those docs any way.  However seeking
+    does act as a short-circuit.  So maybe do some percent of the time or when the level
+    is above some threshold.
+
+  */
+
+    //
+    //  TODO MAJOR REFACTOR SIMPLIFICATION BASED ON TreeCellIterator  TODO
+    //
+
+    private VNode curVNode;//current pointer, derived from query shape
+    private BytesRef curVNodeTerm = new BytesRef();//curVNode.cell's term, without leaf. in main loop only
+
+    private BytesRef thisTerm;//the result of termsEnum.term()
+    private Cell indexedCell;//Cell wrapper of thisTerm. Always updated when thisTerm is.
+
+    public VisitorTemplate(LeafReaderContext context) throws IOException {
+      super(context);
+    }
+
+    public DocIdSet getDocIdSet() throws IOException {
+      assert curVNode == null : "Called more than once?";
+      if (termsEnum == null)
+        return null;
+      if (!nextTerm()) {//advances
+        return null;
+      }
+
+      curVNode = new VNode(null);
+      curVNode.reset(grid.getWorldCell());
+
+      start();
+
+      addIntersectingChildren();
+
+      main: while (thisTerm != null) {//terminates for other reasons too!
+
+        //Advance curVNode pointer
+        if (curVNode.children != null) {
+          //-- HAVE CHILDREN: DESCEND
+          assert curVNode.children.hasNext();//if we put it there then it has something
+          preSiblings(curVNode);
+          curVNode = curVNode.children.next();
+        } else {
+          //-- NO CHILDREN: ADVANCE TO NEXT SIBLING
+          VNode parentVNode = curVNode.parent;
+          while (true) {
+            if (parentVNode == null)
+              break main; // all done
+            if (parentVNode.children.hasNext()) {
+              //advance next sibling
+              curVNode = parentVNode.children.next();
+              break;
+            } else {
+              //reached end of siblings; pop up
+              postSiblings(parentVNode);
+              parentVNode.children = null;//GC
+              parentVNode = parentVNode.parent;
+            }
+          }
+        }
+
+        //Seek to curVNode's cell (or skip if termsEnum has moved beyond)
+        final int compare = indexedCell.compareToNoLeaf(curVNode.cell);
+        if (compare > 0) {
+          // The indexed cell is after; continue loop to next query cell
+          continue;
+        }
+        if (compare < 0) {
+          // The indexed cell is before; seek ahead to query cell:
+          //      Seek !
+          curVNode.cell.getTokenBytesNoLeaf(curVNodeTerm);
+          TermsEnum.SeekStatus seekStatus = termsEnum.seekCeil(curVNodeTerm);
+          if (seekStatus == TermsEnum.SeekStatus.END)
+            break; // all done
+          thisTerm = termsEnum.term();
+          indexedCell = grid.readCell(thisTerm, indexedCell);
+          if (seekStatus == TermsEnum.SeekStatus.NOT_FOUND) {
+            // Did we find a leaf of the cell we were looking for or something after?
+            if (!indexedCell.isLeaf() || indexedCell.compareToNoLeaf(curVNode.cell) != 0)
+              continue; // The indexed cell is after; continue loop to next query cell
+          }
+        }
+        // indexedCell == queryCell (disregarding leaf).
+
+        // If indexedCell is a leaf then there's no prefix (prefix sorts before) -- just visit and continue
+        if (indexedCell.isLeaf()) {
+          visitLeaf(indexedCell);//TODO or query cell? Though shouldn't matter.
+          if (!nextTerm()) break;
+          continue;
+        }
+        // If a prefix (non-leaf) then visit; see if we descend.
+        final boolean descend = visitPrefix(curVNode.cell);//need to use curVNode.cell not indexedCell
+        if (!nextTerm()) break;
+        // Check for adjacent leaf with the same prefix
+        if (indexedCell.isLeaf() && indexedCell.getLevel() == curVNode.cell.getLevel()) {
+          visitLeaf(indexedCell);//TODO or query cell? Though shouldn't matter.
+          if (!nextTerm()) break;
+        }
+
+
+        if (descend) {
+          addIntersectingChildren();
+        }
+
+      }//main loop
+
+      return finish();
+    }
+
+    /** Called initially, and whenever {@link #visitPrefix(org.apache.lucene.spatial.prefix.tree.Cell)}
+     * returns true. */
+    private void addIntersectingChildren() throws IOException {
+      assert thisTerm != null;
+      Cell cell = curVNode.cell;
+      if (cell.getLevel() >= detailLevel)
+        throw new IllegalStateException("Spatial logic error");
+
+      //Decide whether to continue to divide & conquer, or whether it's time to
+      // scan through terms beneath this cell.
+      // Scanning is a performance optimization trade-off.
+
+      //TODO use termsEnum.docFreq() as heuristic
+      boolean scan = cell.getLevel() >= prefixGridScanLevel;//simple heuristic
+
+      if (!scan) {
+        //Divide & conquer (ultimately termsEnum.seek())
+
+        Iterator<Cell> subCellsIter = findSubCellsToVisit(cell);
+        if (!subCellsIter.hasNext())//not expected
+          return;
+        curVNode.children = new VNodeCellIterator(subCellsIter, new VNode(curVNode));
+
+      } else {
+        //Scan (loop of termsEnum.next())
+
+        scan(detailLevel);
+      }
+    }
+
+    /**
+     * Called when doing a divide and conquer to find the next intersecting cells
+     * of the query shape that are beneath {@code cell}. {@code cell} is
+     * guaranteed to have an intersection and thus this must return some number
+     * of nodes.
+     */
+    protected CellIterator findSubCellsToVisit(Cell cell) {
+      return cell.getNextLevelCells(queryShape);
+    }
+
+    /**
+     * Scans ({@code termsEnum.next()}) terms until a term is found that does
+     * not start with curVNode's cell. If it finds a leaf cell or a cell at
+     * level {@code scanDetailLevel} then it calls {@link
+     * #visitScanned(org.apache.lucene.spatial.prefix.tree.Cell)}.
+     */
+    protected void scan(int scanDetailLevel) throws IOException {
+      //note: this can be a do-while instead in 6x; 5x has a back-compat with redundant leaves -- LUCENE-4942
+      while (curVNode.cell.isPrefixOf(indexedCell)) {
+        if (indexedCell.getLevel() == scanDetailLevel
+            || (indexedCell.getLevel() < scanDetailLevel && indexedCell.isLeaf())) {
+          visitScanned(indexedCell);
+        }
+        //advance
+        if (!nextTerm()) break;
+      }
+    }
+
+    private boolean nextTerm() throws IOException {
+      if ((thisTerm = termsEnum.next()) == null)
+        return false;
+      indexedCell = grid.readCell(thisTerm, indexedCell);
+      return true;
+    }
+
+    /** Used for {@link VNode#children}. */
+    private class VNodeCellIterator implements Iterator<VNode> {
+
+      final Iterator<Cell> cellIter;
+      private final VNode vNode;
+
+      VNodeCellIterator(Iterator<Cell> cellIter, VNode vNode) {
+        this.cellIter = cellIter;
+        this.vNode = vNode;
+      }
+
+      @Override
+      public boolean hasNext() {
+        return cellIter.hasNext();
+      }
+
+      @Override
+      public VNode next() {
+        assert hasNext();
+        vNode.reset(cellIter.next());
+        return vNode;
+      }
+
+      @Override
+      public void remove() {//it always removes
+      }
+    }
+
+    /** Called first to setup things. */
+    protected abstract void start() throws IOException;
+
+    /** Called last to return the result. */
+    protected abstract DocIdSet finish() throws IOException;
+
+    /**
+     * Visit an indexed non-leaf cell. The presence of a prefix cell implies
+     * there are leaf cells at further levels. The cell passed should have it's
+     * {@link org.apache.lucene.spatial.prefix.tree.Cell#getShapeRel()} set
+     * relative to the filtered shape.
+     *
+     * @param cell An intersecting cell; not a leaf.
+     * @return true to descend to more levels.
+     */
+    protected abstract boolean visitPrefix(Cell cell) throws IOException;
+
+    /**
+     * Called when an indexed leaf cell is found. An
+     * indexed leaf cell usually means associated documents won't be found at
+     * further detail levels.  However, if a document has
+     * multiple overlapping shapes at different resolutions, then this isn't true.
+     */
+    protected abstract void visitLeaf(Cell cell) throws IOException;
+
+    /**
+     * The cell is either indexed as a leaf or is the last level of detail. It
+     * might not even intersect the query shape, so be sure to check for that.
+     * The default implementation will check that and if passes then call
+     * {@link #visitLeaf(org.apache.lucene.spatial.prefix.tree.Cell)} or
+     * {@link #visitPrefix(org.apache.lucene.spatial.prefix.tree.Cell)}.
+     */
+    protected void visitScanned(Cell cell) throws IOException {
+      final SpatialRelation relate = cell.getShape().relate(queryShape);
+      if (relate.intersects()) {
+        cell.setShapeRel(relate);//just being pedantic
+        if (cell.isLeaf()) {
+          visitLeaf(cell);
+        } else {
+          visitPrefix(cell);
+        }
+      }
+    }
+
+    protected void preSiblings(VNode vNode) throws IOException {
+    }
+
+    protected void postSiblings(VNode vNode) throws IOException {
+    }
+  }//class VisitorTemplate
+
+  /**
+   * A visitor node/cell found via the query shape for {@link VisitorTemplate}.
+   * Sometimes these are reset(cell). It's like a LinkedList node but forms a
+   * tree.
+   *
+   * @lucene.internal
+   */
+  protected static class VNode {
+    //Note: The VNode tree adds more code to debug/maintain v.s. a flattened
+    // LinkedList that we used to have. There is more opportunity here for
+    // custom behavior (see preSiblings & postSiblings) but that's not
+    // leveraged yet. Maybe this is slightly more GC friendly.
+
+    final VNode parent;//only null at the root
+    Iterator<VNode> children;//null, then sometimes set, then null
+    Cell cell;//not null (except initially before reset())
+
+    /**
+     * call reset(cell) after to set the cell.
+     */
+    VNode(VNode parent) { // remember to call reset(cell) after
+      this.parent = parent;
+    }
+
+    void reset(Cell cell) {
+      assert cell != null;
+      this.cell = cell;
+      assert children == null;
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/BytesRefIteratorTokenStream.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/BytesRefIteratorTokenStream.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/BytesRefIteratorTokenStream.java
new file mode 100644
index 0000000..e724ab0
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/BytesRefIteratorTokenStream.java
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.BytesTermAttribute;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefIterator;
+
+/**
+ * A TokenStream used internally by {@link org.apache.lucene.spatial.prefix.PrefixTreeStrategy}.
+ *
+ * This is modelled after {@link org.apache.lucene.analysis.LegacyNumericTokenStream}.
+ *
+ * @lucene.internal
+ */
+class BytesRefIteratorTokenStream extends TokenStream {
+
+  public BytesRefIterator getBytesRefIterator() {
+    return bytesIter;
+  }
+
+  public BytesRefIteratorTokenStream setBytesRefIterator(BytesRefIterator iter) {
+    this.bytesIter = iter;
+    return this;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    if (bytesIter == null)
+      throw new IllegalStateException("call setBytesRefIterator() before usage");
+  }
+
+  @Override
+  public final boolean incrementToken() throws IOException {
+    if (bytesIter == null)
+      throw new IllegalStateException("call setBytesRefIterator() before usage");
+
+    // get next
+    BytesRef bytes = bytesIter.next();
+    if (bytes == null) {
+      return false;
+    } else {
+      clearAttributes();
+      bytesAtt.setBytesRef(bytes);
+      //note: we don't bother setting posInc or type attributes.  There's no point to it.
+      return true;
+    }
+  }
+
+  //members
+  private final BytesTermAttribute bytesAtt = addAttribute(BytesTermAttribute.class);
+
+  private BytesRefIterator bytesIter = null; // null means not initialized
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/CellToBytesRefIterator.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/CellToBytesRefIterator.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/CellToBytesRefIterator.java
new file mode 100644
index 0000000..0b81b26
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/CellToBytesRefIterator.java
@@ -0,0 +1,49 @@
+/*
+ * 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;
+
+import java.util.Iterator;
+
+import org.apache.lucene.spatial.prefix.tree.Cell;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefIterator;
+
+/**
+ * A reset'able {@link org.apache.lucene.util.BytesRefIterator} wrapper around
+ * an {@link java.util.Iterator} of {@link org.apache.lucene.spatial.prefix.tree.Cell}s.
+ *
+ * @see PrefixTreeStrategy#newCellToBytesRefIterator()
+ *
+ * @lucene.internal
+ */
+public class CellToBytesRefIterator implements BytesRefIterator {
+
+  protected Iterator<Cell> cellIter;
+  protected BytesRef bytesRef = new BytesRef();
+
+  public void reset(Iterator<Cell> cellIter) {
+    this.cellIter = cellIter;
+  }
+
+  @Override
+  public BytesRef next() {
+    if (!cellIter.hasNext()) {
+      return null;
+    }
+    return cellIter.next().getTokenBytesWithLeaf(bytesRef);
+  }
+}