You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ho...@apache.org on 2016/03/01 18:07:14 UTC

[31/50] [abbrv] 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/prefix/ContainsPrefixTreeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeQuery.java
new file mode 100644
index 0000000..0046378
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeQuery.java
@@ -0,0 +1,362 @@
+/*
+ * 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.Arrays;
+
+import com.spatial4j.core.shape.Shape;
+import com.spatial4j.core.shape.SpatialRelation;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.DocIdSetIterator;
+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.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.SentinelIntSet;
+
+/**
+ * Finds docs where its indexed shape {@link org.apache.lucene.spatial.query.SpatialOperation#Contains
+ * CONTAINS} the query shape. For use on {@link RecursivePrefixTreeStrategy}.
+ *
+ * @lucene.experimental
+ */
+public class ContainsPrefixTreeQuery extends AbstractPrefixTreeQuery {
+
+  /**
+   * If the spatial data for a document is comprised of multiple overlapping or adjacent parts,
+   * it might fail to match a query shape when doing the CONTAINS predicate when the sum of
+   * those shapes contain the query shape but none do individually.  Set this to false to
+   * increase performance if you don't care about that circumstance (such as if your indexed
+   * data doesn't even have such conditions).  See LUCENE-5062.
+   */
+  protected final boolean multiOverlappingIndexedShapes;
+
+  public ContainsPrefixTreeQuery(Shape queryShape, String fieldName, SpatialPrefixTree grid, int detailLevel, boolean multiOverlappingIndexedShapes) {
+    super(queryShape, fieldName, grid, detailLevel);
+    this.multiOverlappingIndexedShapes = multiOverlappingIndexedShapes;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (!super.equals(o))
+      return false;
+    return multiOverlappingIndexedShapes == ((ContainsPrefixTreeQuery)o).multiOverlappingIndexedShapes;
+  }
+
+  @Override
+  public int hashCode() {
+    return super.hashCode() + (multiOverlappingIndexedShapes ? 1 : 0);
+  }
+
+  @Override
+  public String toString(String field) {
+    return getClass().getSimpleName() + "(" +
+        "fieldName=" + fieldName + "," +
+        "queryShape=" + queryShape + "," +
+        "detailLevel=" + detailLevel + "," +
+        "multiOverlappingIndexedShapes=" + multiOverlappingIndexedShapes +
+        ")";
+  }
+
+  @Override
+  protected DocIdSet getDocIdSet(LeafReaderContext context) throws IOException {
+    return new ContainsVisitor(context).visit(grid.getWorldCell(), null);
+  }
+
+  private class ContainsVisitor extends BaseTermsEnumTraverser {
+
+    public ContainsVisitor(LeafReaderContext context) throws IOException {
+      super(context);
+      if (termsEnum != null) {
+        nextTerm();//advance to first
+      }
+    }
+
+    BytesRef seekTerm = new BytesRef();//temp; see seek()
+    BytesRef thisTerm;//current term in termsEnum
+    Cell indexedCell;//the cell wrapper around thisTerm
+
+    /** This is the primary algorithm; recursive.  Returns null if finds none. */
+    private SmallDocSet visit(Cell cell, Bits acceptContains) throws IOException {
+
+      if (thisTerm == null)//signals all done
+        return null;
+
+      // Get the AND of all child results (into combinedSubResults)
+      SmallDocSet combinedSubResults = null;
+      //   Optimization: use null subCellsFilter when we know cell is within the query shape.
+      Shape subCellsFilter = queryShape;
+      if (cell.getLevel() != 0 && ((cell.getShapeRel() == null || cell.getShapeRel() == SpatialRelation.WITHIN))) {
+        subCellsFilter = null;
+        assert cell.getShape().relate(queryShape) == SpatialRelation.WITHIN;
+      }
+      CellIterator subCells = cell.getNextLevelCells(subCellsFilter);
+      while (subCells.hasNext()) {
+        Cell subCell = subCells.next();
+        if (!seek(subCell)) {
+          combinedSubResults = null;
+        } else if (subCell.getLevel() == detailLevel) {
+          combinedSubResults = getDocs(subCell, acceptContains);
+        } else if (!multiOverlappingIndexedShapes &&
+            subCell.getShapeRel() == SpatialRelation.WITHIN) {
+          combinedSubResults = getLeafDocs(subCell, acceptContains);
+        } else {
+          //OR the leaf docs with all child results
+          SmallDocSet leafDocs = getLeafDocs(subCell, acceptContains);
+          SmallDocSet subDocs = visit(subCell, acceptContains); //recursion
+          combinedSubResults = union(leafDocs, subDocs);
+        }
+
+        if (combinedSubResults == null)
+          break;
+        acceptContains = combinedSubResults;//has the 'AND' effect on next iteration
+      }
+
+      return combinedSubResults;
+    }
+
+    private boolean seek(Cell cell) throws IOException {
+      if (thisTerm == null)
+        return false;
+      final int compare = indexedCell.compareToNoLeaf(cell);
+      if (compare > 0) {
+        return false;//leap-frog effect
+      } else if (compare == 0) {
+        return true; // already there!
+      } else {//compare > 0
+        //seek!
+        seekTerm = cell.getTokenBytesNoLeaf(seekTerm);
+        final TermsEnum.SeekStatus seekStatus = termsEnum.seekCeil(seekTerm);
+        if (seekStatus == TermsEnum.SeekStatus.END) {
+          thisTerm = null;//all done
+          return false;
+        }
+        thisTerm = termsEnum.term();
+        indexedCell = grid.readCell(thisTerm, indexedCell);
+        if (seekStatus == TermsEnum.SeekStatus.FOUND) {
+          return true;
+        }
+        return indexedCell.isLeaf() && indexedCell.compareToNoLeaf(cell) == 0;
+      }
+    }
+
+    /** Get prefix & leaf docs at this cell. */
+    private SmallDocSet getDocs(Cell cell, Bits acceptContains) throws IOException {
+      assert indexedCell.compareToNoLeaf(cell) == 0;
+      //called when we've reached detailLevel.
+      if (indexedCell.isLeaf()) {//only a leaf
+        SmallDocSet result = collectDocs(acceptContains);
+        nextTerm();
+        return result;
+      } else {
+        SmallDocSet docsAtPrefix = collectDocs(acceptContains);
+        if (!nextTerm()) {
+          return docsAtPrefix;
+        }
+        //collect leaf too
+        if (indexedCell.isLeaf() && indexedCell.compareToNoLeaf(cell) == 0) {
+          SmallDocSet docsAtLeaf = collectDocs(acceptContains);
+          nextTerm();
+          return union(docsAtPrefix, docsAtLeaf);
+        } else {
+          return docsAtPrefix;
+        }
+      }
+    }
+
+    /** Gets docs on the leaf of the given cell, _if_ there is a leaf cell, otherwise null. */
+    private SmallDocSet getLeafDocs(Cell cell, Bits acceptContains) throws IOException {
+      assert indexedCell.compareToNoLeaf(cell) == 0;
+      //Advance past prefix if we're at a prefix; return null if no leaf
+      if (!indexedCell.isLeaf()) {
+        if (!nextTerm() || !indexedCell.isLeaf() || indexedCell.getLevel() != cell.getLevel()) {
+          return null;
+        }
+      }
+      SmallDocSet result = collectDocs(acceptContains);
+      nextTerm();
+      return result;
+    }
+
+    private boolean nextTerm() throws IOException {
+      if ((thisTerm = termsEnum.next()) == null)
+        return false;
+      indexedCell = grid.readCell(thisTerm, indexedCell);
+      return true;
+    }
+
+    private SmallDocSet union(SmallDocSet aSet, SmallDocSet bSet) {
+      if (bSet != null) {
+        if (aSet == null)
+          return bSet;
+        return aSet.union(bSet);//union is 'or'
+      }
+      return aSet;
+    }
+
+    private SmallDocSet collectDocs(Bits acceptContains) throws IOException {
+      SmallDocSet set = null;
+
+      postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
+      int docid;
+      while ((docid = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+        if (acceptContains != null && acceptContains.get(docid) == false) {
+          continue;
+        }
+        if (set == null) {
+          int size = termsEnum.docFreq();
+          if (size <= 0)
+            size = 16;
+          set = new SmallDocSet(size);
+        }
+        set.set(docid);
+      }
+      return set;
+    }
+
+  }//class ContainsVisitor
+
+  /** A hash based mutable set of docIds. If this were Solr code then we might
+   * use a combination of HashDocSet and SortedIntDocSet instead. */
+  // TODO use DocIdSetBuilder?
+  private static class SmallDocSet extends DocIdSet implements Bits {
+
+    private final SentinelIntSet intSet;
+    private int maxInt = 0;
+
+    public SmallDocSet(int size) {
+      intSet = new SentinelIntSet(size, -1);
+    }
+
+    @Override
+    public boolean get(int index) {
+      return intSet.exists(index);
+    }
+
+    public void set(int index) {
+      intSet.put(index);
+      if (index > maxInt)
+        maxInt = index;
+    }
+
+    /** Largest docid. */
+    @Override
+    public int length() {
+      return maxInt;
+    }
+
+    /** Number of docids. */
+    public int size() {
+      return intSet.size();
+    }
+
+    /** NOTE: modifies and returns either "this" or "other" */
+    public SmallDocSet union(SmallDocSet other) {
+      SmallDocSet bigger;
+      SmallDocSet smaller;
+      if (other.intSet.size() > this.intSet.size()) {
+        bigger = other;
+        smaller = this;
+      } else {
+        bigger = this;
+        smaller = other;
+      }
+      //modify bigger
+      for (int v : smaller.intSet.keys) {
+        if (v == smaller.intSet.emptyVal)
+          continue;
+        bigger.set(v);
+      }
+      return bigger;
+    }
+
+    @Override
+    public Bits bits() throws IOException {
+      //if the # of docids is super small, return null since iteration is going
+      // to be faster
+      return size() > 4 ? this : null;
+    }
+
+    @Override
+    public DocIdSetIterator iterator() throws IOException {
+      if (size() == 0)
+        return null;
+      //copy the unsorted values to a new array then sort them
+      int d = 0;
+      final int[] docs = new int[intSet.size()];
+      for (int v : intSet.keys) {
+        if (v == intSet.emptyVal)
+          continue;
+        docs[d++] = v;
+      }
+      assert d == intSet.size();
+      final int size = d;
+
+      //sort them
+      Arrays.sort(docs, 0, size);
+
+      return new DocIdSetIterator() {
+        int idx = -1;
+        @Override
+        public int docID() {
+          if (idx < 0) {
+            return -1;
+          } else if (idx < size) {
+            return docs[idx];
+          } else {
+            return NO_MORE_DOCS;
+          }
+        }
+
+        @Override
+        public int nextDoc() throws IOException {
+          if (++idx < size)
+            return docs[idx];
+          return NO_MORE_DOCS;
+        }
+
+        @Override
+        public int advance(int target) throws IOException {
+          //for this small set this is likely faster vs. a binary search
+          // into the sorted array
+          return slowAdvance(target);
+        }
+
+        @Override
+        public long cost() {
+          return size;
+        }
+      };
+    }
+
+    @Override
+    public long ramBytesUsed() {
+      return RamUsageEstimator.alignObjectSize(
+            RamUsageEstimator.NUM_BYTES_OBJECT_REF
+          + Integer.BYTES)
+          + intSet.ramBytesUsed();
+    }
+
+  }//class SmallDocSet
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/HeatmapFacetCounter.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/HeatmapFacetCounter.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/HeatmapFacetCounter.java
new file mode 100644
index 0000000..c6700cd
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/HeatmapFacetCounter.java
@@ -0,0 +1,310 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+
+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 com.spatial4j.core.shape.SpatialRelation;
+import org.apache.lucene.index.IndexReaderContext;
+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.ArrayUtil;
+import org.apache.lucene.util.Bits;
+
+/**
+ * Computes spatial facets in two dimensions as a grid of numbers.  The data is often visualized as a so-called
+ * "heatmap", hence the name.
+ *
+ * @lucene.experimental
+ */
+public class HeatmapFacetCounter {
+  //TODO where should this code live? It could go to PrefixTreeFacetCounter, or maybe here in its own class is fine.
+
+  /** Maximum number of supported rows (or columns). */
+  public static final int MAX_ROWS_OR_COLUMNS = (int) Math.sqrt(ArrayUtil.MAX_ARRAY_LENGTH);
+  static {
+    Math.multiplyExact(MAX_ROWS_OR_COLUMNS, MAX_ROWS_OR_COLUMNS);//will throw if doesn't stay within integer
+  }
+
+  /** Response structure */
+  public static class Heatmap {
+    public final int columns;
+    public final int rows;
+    public final int[] counts;//in order of 1st column (all rows) then 2nd column (all rows) etc.
+    public final Rectangle region;
+
+    public Heatmap(int columns, int rows, Rectangle region) {
+      this.columns = columns;
+      this.rows = rows;
+      this.counts = new int[columns * rows];
+      this.region = region;
+    }
+
+    public int getCount(int x, int y) {
+      return counts[x * rows + y];
+    }
+
+    @Override
+    public String toString() {
+      return "Heatmap{" + columns + "x" + rows + " " + region + '}';
+    }
+  }
+
+  /**
+   * Calculates spatial 2D facets (aggregated counts) in a grid, sometimes called a heatmap.
+   * Facet computation is implemented by navigating the underlying indexed terms efficiently. If you don't know exactly
+   * what facetLevel to go to for a given input box but you have some sense of how many cells there should be relative
+   * to the size of the shape, then consider using the logic that {@link org.apache.lucene.spatial.prefix.PrefixTreeStrategy}
+   * uses when approximating what level to go to when indexing a shape given a distErrPct.
+   *
+   * @param context the IndexReader's context
+   * @param topAcceptDocs a Bits to limit counted docs.  If null, live docs are counted.
+   * @param inputShape the shape to gather grid squares for; typically a {@link Rectangle}.
+   *                   The <em>actual</em> heatmap area will usually be larger since the cells on the edge that overlap
+   *                   are returned. We always return a rectangle of integers even if the inputShape isn't a rectangle
+   *                   -- the non-intersecting cells will all be 0.
+   *                   If null is given, the entire world is assumed.
+   * @param facetLevel the target depth (detail) of cells.
+   * @param maxCells the maximum number of cells to return. If the cells exceed this count, an
+   */
+  public static Heatmap calcFacets(PrefixTreeStrategy strategy, IndexReaderContext context, Bits topAcceptDocs,
+                                   Shape inputShape, final int facetLevel, int maxCells) throws IOException {
+    if (maxCells > (MAX_ROWS_OR_COLUMNS * MAX_ROWS_OR_COLUMNS)) {
+      throw new IllegalArgumentException("maxCells (" + maxCells + ") should be <= " + MAX_ROWS_OR_COLUMNS);
+    }
+    if (inputShape == null) {
+      inputShape = strategy.getSpatialContext().getWorldBounds();
+    }
+    final Rectangle inputRect = inputShape.getBoundingBox();
+    //First get the rect of the cell at the bottom-left at depth facetLevel
+    final SpatialPrefixTree grid = strategy.getGrid();
+    final SpatialContext ctx = grid.getSpatialContext();
+    final Point cornerPt = ctx.makePoint(inputRect.getMinX(), inputRect.getMinY());
+    final CellIterator cellIterator = grid.getTreeCellIterator(cornerPt, facetLevel);
+    Cell cornerCell = null;
+    while (cellIterator.hasNext()) {
+      cornerCell = cellIterator.next();
+    }
+    assert cornerCell != null && cornerCell.getLevel() == facetLevel : "Cell not at target level: " + cornerCell;
+    final Rectangle cornerRect = (Rectangle) cornerCell.getShape();
+    assert cornerRect.hasArea();
+    //Now calculate the number of columns and rows necessary to cover the inputRect
+    double heatMinX = cornerRect.getMinX();//note: we might change this below...
+    final double cellWidth = cornerRect.getWidth();
+    final Rectangle worldRect = ctx.getWorldBounds();
+    final int columns = calcRowsOrCols(cellWidth, heatMinX, inputRect.getWidth(), inputRect.getMinX(), worldRect.getWidth());
+    final double heatMinY = cornerRect.getMinY();
+    final double cellHeight = cornerRect.getHeight();
+    final int rows = calcRowsOrCols(cellHeight, heatMinY, inputRect.getHeight(), inputRect.getMinY(), worldRect.getHeight());
+    assert rows > 0 && columns > 0;
+    if (columns > MAX_ROWS_OR_COLUMNS || rows > MAX_ROWS_OR_COLUMNS || columns * rows > maxCells) {
+      throw new IllegalArgumentException(
+          "Too many cells (" + columns + " x " + rows + ") for level " + facetLevel + " shape " + inputRect);
+    }
+
+    //Create resulting heatmap bounding rectangle & Heatmap object.
+    final double halfCellWidth = cellWidth / 2.0;
+    // if X world-wraps, use world bounds' range
+    if (columns * cellWidth + halfCellWidth > worldRect.getWidth()) {
+      heatMinX = worldRect.getMinX();
+    }
+    double heatMaxX = heatMinX + columns * cellWidth;
+    if (Math.abs(heatMaxX - worldRect.getMaxX()) < halfCellWidth) {//numeric conditioning issue
+      heatMaxX = worldRect.getMaxX();
+    } else if (heatMaxX > worldRect.getMaxX()) {//wraps dateline (won't happen if !geo)
+      heatMaxX = heatMaxX - worldRect.getMaxX() +  worldRect.getMinX();
+    }
+    final double halfCellHeight = cellHeight / 2.0;
+    double heatMaxY = heatMinY + rows * cellHeight;
+    if (Math.abs(heatMaxY - worldRect.getMaxY()) < halfCellHeight) {//numeric conditioning issue
+      heatMaxY = worldRect.getMaxY();
+    }
+
+    final Heatmap heatmap = new Heatmap(columns, rows, ctx.makeRectangle(heatMinX, heatMaxX, heatMinY, heatMaxY));
+
+    //All ancestor cell counts (of facetLevel) will be captured during facet visiting and applied later. If the data is
+    // just points then there won't be any ancestors.
+    //Facet count of ancestors covering all of the heatmap:
+    int[] allCellsAncestorCount = new int[1]; // single-element array so it can be accumulated in the inner class
+    //All other ancestors:
+    Map<Rectangle,Integer> ancestors = new HashMap<>();
+
+    //Now lets count some facets!
+    PrefixTreeFacetCounter.compute(strategy, context, topAcceptDocs, inputShape, facetLevel,
+        new PrefixTreeFacetCounter.FacetVisitor() {
+      @Override
+      public void visit(Cell cell, int count) {
+        final double heatMinX = heatmap.region.getMinX();
+        final Rectangle rect = (Rectangle) cell.getShape();
+        if (cell.getLevel() == facetLevel) {//heatmap level; count it directly
+          //convert to col & row
+          int column;
+          if (rect.getMinX() >= heatMinX) {
+            column = (int) Math.round((rect.getMinX() - heatMinX) / cellWidth);
+          } else { // due to dateline wrap
+            column = (int) Math.round((rect.getMinX() + 360 - heatMinX) / cellWidth);
+          }
+          int row = (int) Math.round((rect.getMinY() - heatMinY) / cellHeight);
+          //note: unfortunately, it's possible for us to visit adjacent cells to the heatmap (if the SpatialPrefixTree
+          // allows adjacent cells to overlap on the seam), so we need to skip them
+          if (column < 0 || column >= heatmap.columns || row < 0 || row >= heatmap.rows) {
+            return;
+          }
+          // increment
+          heatmap.counts[column * heatmap.rows + row] += count;
+
+        } else if (rect.relate(heatmap.region) == SpatialRelation.CONTAINS) {//containing ancestor
+          allCellsAncestorCount[0] += count;
+
+        } else { // ancestor
+          // note: not particularly efficient (possible put twice, and Integer wrapper); oh well
+          Integer existingCount = ancestors.put(rect, count);
+          if (existingCount != null) {
+            ancestors.put(rect, count + existingCount);
+          }
+        }
+      }
+    });
+
+    //Update the heatmap counts with ancestor counts
+
+    // Apply allCellsAncestorCount
+    if (allCellsAncestorCount[0] > 0) {
+      for (int i = 0; i < heatmap.counts.length; i++) {
+        heatmap.counts[i] += allCellsAncestorCount[0];
+      }
+    }
+
+    // Apply ancestors
+    //  note: This approach isn't optimized for a ton of ancestor cells. We'll potentially increment the same cells
+    //    multiple times in separate passes if any ancestors overlap. IF this poses a problem, we could optimize it
+    //    with additional complication by keeping track of intervals in a sorted tree structure (possible TreeMap/Set)
+    //    and iterate them cleverly such that we just make one pass at this stage.
+
+    int[] pair = new int[2];//output of intersectInterval
+    for (Map.Entry<Rectangle, Integer> entry : ancestors.entrySet()) {
+      Rectangle rect = entry.getKey();
+      final int count = entry.getValue();
+      //note: we approach this in a way that eliminates int overflow/underflow (think huge cell, tiny heatmap)
+      intersectInterval(heatMinY, heatMaxY, cellHeight, rows, rect.getMinY(), rect.getMaxY(), pair);
+      final int startRow = pair[0];
+      final int endRow = pair[1];
+
+      if (!heatmap.region.getCrossesDateLine()) {
+        intersectInterval(heatMinX, heatMaxX, cellWidth, columns, rect.getMinX(), rect.getMaxX(), pair);
+        final int startCol = pair[0];
+        final int endCol = pair[1];
+        incrementRange(heatmap, startCol, endCol, startRow, endRow, count);
+
+      } else {
+        //left half of dateline:
+        if (rect.getMaxX() >= heatMinX) {
+          final int leftColumns = (int) Math.round((180 - heatMinX) / cellWidth) + 1;
+          intersectInterval(heatMinX, 180, cellWidth, leftColumns, rect.getMinX(), rect.getMaxX(), pair);
+          final int startCol = pair[0];
+          final int endCol = pair[1];
+          incrementRange(heatmap, startCol, endCol, startRow, endRow, count);
+        }
+        //right half of dateline
+        if (rect.getMinY() <= heatMaxX) {
+          final int rightColumns = (int) Math.round(heatMaxX / cellWidth) + 1;
+          intersectInterval(0, heatMaxX, cellWidth, rightColumns, rect.getMinX(), rect.getMaxX(), pair);
+          final int startCol = pair[0];
+          final int endCol = pair[1];
+          incrementRange(heatmap, startCol, endCol, startRow, endRow, count);
+        }
+      }
+
+    }
+
+    return heatmap;
+  }
+
+  private static void intersectInterval(double heatMin, double heatMax, double heatCellLen, int heatLen,
+                                        double cellMin, double cellMax,
+                                        int[] out) {
+    //precondition: we know there's an intersection
+    if (heatMin >= cellMin) {
+      out[0] = 0;
+    } else {
+      out[0] = (int) Math.round((cellMin - heatMin) / heatCellLen);
+    }
+    if (heatMax <= cellMax) {
+      out[1] = heatLen - 1;
+    } else {
+      out[1] = (int) Math.round((cellMax - heatMin) / heatCellLen) - 1;
+    }
+  }
+
+  private static void incrementRange(Heatmap heatmap, int startColumn, int endColumn, int startRow, int endRow,
+                                     int count) {
+    //startColumn & startRow are not necessarily within the heatmap range; likewise numRows/columns may overlap.
+    if (startColumn < 0) {
+      endColumn += startColumn;
+      startColumn = 0;
+    }
+    endColumn = Math.min(heatmap.columns-1, endColumn);
+
+    if (startRow < 0) {
+      endRow += startRow;
+      startRow = 0;
+    }
+    endRow = Math.min(heatmap.rows-1, endRow);
+
+    if (startRow > endRow) {
+      return;//short-circuit
+    }
+    for (int c = startColumn; c <= endColumn; c++) {
+      int cBase = c * heatmap.rows;
+      for (int r = startRow; r <= endRow; r++) {
+        heatmap.counts[cBase + r] += count;
+      }
+    }
+  }
+
+  /** Computes the number of intervals (rows or columns) to cover a range given the sizes. */
+  private static int calcRowsOrCols(double cellRange, double cellMin, double requestRange, double requestMin,
+                                    double worldRange) {
+    assert requestMin >= cellMin;
+    //Idealistically this wouldn't be so complicated but we concern ourselves with overflow and edge cases
+    double range = (requestRange + (requestMin - cellMin));
+    if (range == 0) {
+      return 1;
+    }
+    final double intervals = Math.ceil(range / cellRange);
+    if (intervals > Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;//should result in an error soon (exceed thresholds)
+    }
+    // ensures we don't have more intervals than world bounds (possibly due to rounding/edge issue)
+    final long intervalsMax = Math.round(worldRange / cellRange);
+    if (intervalsMax > Integer.MAX_VALUE) {
+      //just return intervals
+      return (int) intervals;
+    }
+    return Math.min((int)intervalsMax, (int)intervals);
+  }
+
+  private HeatmapFacetCounter() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/IntersectsPrefixTreeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/IntersectsPrefixTreeQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/IntersectsPrefixTreeQuery.java
new file mode 100644
index 0000000..ccb0f89
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/IntersectsPrefixTreeQuery.java
@@ -0,0 +1,95 @@
+/*
+ * 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 com.spatial4j.core.shape.SpatialRelation;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.spatial.prefix.tree.Cell;
+import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
+import org.apache.lucene.util.BitDocIdSet;
+import org.apache.lucene.util.FixedBitSet;
+
+/**
+ * A Query matching documents that have an {@link SpatialRelation#INTERSECTS}
+ * (i.e. not DISTINCT) relationship with a provided query shape.
+ *
+ * @lucene.internal
+ */
+public class IntersectsPrefixTreeQuery extends AbstractVisitingPrefixTreeQuery {
+
+  public IntersectsPrefixTreeQuery(Shape queryShape, String fieldName,
+                                   SpatialPrefixTree grid, int detailLevel,
+                                   int prefixGridScanLevel) {
+    super(queryShape, fieldName, grid, detailLevel, prefixGridScanLevel);
+  }
+
+  @Override
+  protected DocIdSet getDocIdSet(LeafReaderContext context) throws IOException {
+    /* Possible optimizations (in IN ADDITION TO THOSE LISTED IN VISITORTEMPLATE):
+
+    * If docFreq is 1 (or < than some small threshold), then check to see if we've already
+      collected it; if so short-circuit. Don't do this just for point data, as there is
+      no benefit, or only marginal benefit when multi-valued.
+
+    * Point query shape optimization when the only indexed data is a point (no leaves).  Result is a term query.
+
+     */
+    return new VisitorTemplate(context) {
+      private FixedBitSet results;
+
+      @Override
+      protected void start() {
+        results = new FixedBitSet(maxDoc);
+      }
+
+      @Override
+      protected DocIdSet finish() {
+        return new BitDocIdSet(results);
+      }
+
+      @Override
+      protected boolean visitPrefix(Cell cell) throws IOException {
+        if (cell.getShapeRel() == SpatialRelation.WITHIN || cell.getLevel() == detailLevel) {
+          collectDocs(results);
+          return false;
+        }
+        return true;
+      }
+
+      @Override
+      protected void visitLeaf(Cell cell) throws IOException {
+        collectDocs(results);
+      }
+
+    }.getDocIdSet();
+  }
+
+  @Override
+  public String toString(String field) {
+    return getClass().getSimpleName() + "(" +
+        "fieldName=" + fieldName + "," +
+        "queryShape=" + queryShape + "," +
+        "detailLevel=" + detailLevel + "," +
+        "prefixGridScanLevel=" + prefixGridScanLevel +
+        ")";
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java
new file mode 100644
index 0000000..8001c82
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java
@@ -0,0 +1,199 @@
+/*
+ * 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.Arrays;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import com.spatial4j.core.shape.Point;
+import com.spatial4j.core.shape.Shape;
+import org.apache.lucene.index.IndexReaderContext;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.spatial.prefix.tree.Cell;
+import org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree;
+import org.apache.lucene.util.Bits;
+
+import static org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape;
+
+/** A PrefixTree based on Number/Date ranges. This isn't very "spatial" on the surface (to the user) but
+ * it's implemented using spatial so that's why it's here extending a SpatialStrategy. When using this class, you will
+ * use various utility methods on the prefix tree implementation to convert objects/strings to/from shapes.
+ *
+ * To use with dates, pass in {@link org.apache.lucene.spatial.prefix.tree.DateRangePrefixTree}.
+ *
+ * @lucene.experimental
+ */
+public class NumberRangePrefixTreeStrategy extends RecursivePrefixTreeStrategy {
+
+  public NumberRangePrefixTreeStrategy(NumberRangePrefixTree prefixTree, String fieldName) {
+    super(prefixTree, fieldName);
+    setPruneLeafyBranches(false);
+    setPrefixGridScanLevel(prefixTree.getMaxLevels()-2);//user might want to change, however
+    setPointsOnly(false);
+    setDistErrPct(0);
+  }
+
+  @Override
+  public NumberRangePrefixTree getGrid() {
+    return (NumberRangePrefixTree) super.getGrid();
+  }
+
+  @Override
+  protected Iterator<Cell> createCellIteratorToIndex(Shape shape, int detailLevel, Iterator<Cell> reuse) {
+    //levels doesn't actually matter; NumberRange based Shapes have their own "level".
+    return super.createCellIteratorToIndex(shape, grid.getMaxLevels(), reuse);
+  }
+
+  /** Unsupported. */
+  @Override
+  public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
+    throw new UnsupportedOperationException();
+  }
+
+  /** Calculates facets between {@code start} and {@code end} to a detail level one greater than that provided by the
+   * arguments. For example providing March to October of 2014 would return facets to the day level of those months.
+   * This is just a convenience method.
+   * @see #calcFacets(IndexReaderContext, Bits, Shape, int)
+   */
+  public Facets calcFacets(IndexReaderContext context, Bits topAcceptDocs, UnitNRShape start, UnitNRShape end)
+      throws IOException {
+    Shape facetRange = getGrid().toRangeShape(start, end);
+    int detailLevel = Math.max(start.getLevel(), end.getLevel()) + 1;
+    return calcFacets(context, topAcceptDocs, facetRange, detailLevel);
+  }
+
+  /**
+   * Calculates facets (aggregated counts) given a range shape (start-end span) and a level, which specifies the detail.
+   * To get the level of an existing shape, say a Calendar, call
+   * {@link org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree#toUnitShape(Object)} then call
+   * {@link org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape#getLevel()}.
+   * Facet computation is implemented by navigating the underlying indexed terms efficiently.
+   */
+  public Facets calcFacets(IndexReaderContext context, Bits topAcceptDocs, Shape facetRange, final int level)
+      throws IOException {
+    final Facets facets = new Facets(level);
+    PrefixTreeFacetCounter.compute(this, context, topAcceptDocs, facetRange, level,
+        new PrefixTreeFacetCounter.FacetVisitor() {
+          Facets.FacetParentVal parentFacet;
+          UnitNRShape parentShape;
+
+          @Override
+          public void visit(Cell cell, int count) {
+            if (cell.getLevel() < level - 1) {//some ancestor of parent facet level, direct or distant
+              parentFacet = null;//reset
+              parentShape = null;//reset
+              facets.topLeaves += count;
+            } else if (cell.getLevel() == level - 1) {//parent
+              //set up FacetParentVal
+              setupParent((UnitNRShape) cell.getShape());
+              parentFacet.parentLeaves += count;
+            } else {//at facet level
+              UnitNRShape unitShape = (UnitNRShape) cell.getShape();
+              UnitNRShape unitShapeParent = unitShape.getShapeAtLevel(unitShape.getLevel() - 1);
+              if (parentFacet == null || !parentShape.equals(unitShapeParent)) {
+                setupParent(unitShapeParent);
+              }
+              //lazy init childCounts
+              if (parentFacet.childCounts == null) {
+                parentFacet.childCounts = new int[parentFacet.childCountsLen];
+              }
+              parentFacet.childCounts[unitShape.getValAtLevel(cell.getLevel())] += count;
+            }
+          }
+
+          private void setupParent(UnitNRShape unitShape) {
+            parentShape = unitShape.clone();
+            //Look for existing parentFacet (from previous segment), or create anew if needed
+            parentFacet = facets.parents.get(parentShape);
+            if (parentFacet == null) {//didn't find one; make a new one
+              parentFacet = new Facets.FacetParentVal();
+              parentFacet.childCountsLen = getGrid().getNumSubCells(parentShape);
+              facets.parents.put(parentShape, parentFacet);
+            }
+          }
+        });
+    return facets;
+  }
+
+  /** Facet response information */
+  public static class Facets {
+    //TODO consider a variable-level structure -- more general purpose.
+
+    public Facets(int detailLevel) {
+      this.detailLevel = detailLevel;
+    }
+
+    /** The bottom-most detail-level counted, as requested. */
+    public final int detailLevel;
+
+    /**
+     * The count of documents with ranges that completely spanned the parents of the detail level. In more technical
+     * terms, this is the count of leaf cells 2 up and higher from the bottom. Usually you only care about counts at
+     * detailLevel, and so you will add this number to all other counts below, including to omitted/implied children
+     * counts of 0. If there are no indexed ranges (just instances, i.e. fully specified dates) then this value will
+     * always be 0.
+     */
+    public int topLeaves;
+
+    /** Holds all the {@link FacetParentVal} instances in order of the key. This is sparse; there won't be an
+     * instance if it's count and children are all 0. The keys are {@link org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape} shapes, which can be
+     * converted back to the original Object (i.e. a Calendar) via
+     * {@link NumberRangePrefixTree#toObject(org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape)}. */
+    public final SortedMap<UnitNRShape,FacetParentVal> parents = new TreeMap<>();
+
+    /** Holds a block of detailLevel counts aggregated to their parent level. */
+    public static class FacetParentVal {
+
+      /** The count of ranges that span all of the childCounts.  In more technical terms, this is the number of leaf
+       * cells found at this parent.  Treat this like {@link Facets#topLeaves}. */
+      public int parentLeaves;
+
+      /** The length of {@link #childCounts}. If childCounts is not null then this is childCounts.length, otherwise it
+       * says how long it would have been if it weren't null. */
+      public int childCountsLen;
+
+      /** The detail level counts. It will be null if there are none, and thus they are assumed 0. Most apps, when
+       * presenting the information, will add {@link #topLeaves} and {@link #parentLeaves} to each count. */
+      public int[] childCounts;
+      //assert childCountsLen == childCounts.length
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder buf = new StringBuilder(2048);
+      buf.append("Facets: level=" + detailLevel + " topLeaves=" + topLeaves + " parentCount=" + parents.size());
+      for (Map.Entry<UnitNRShape, FacetParentVal> entry : parents.entrySet()) {
+        buf.append('\n');
+        if (buf.length() > 1000) {
+          buf.append("...");
+          break;
+        }
+        final FacetParentVal pVal = entry.getValue();
+        buf.append(' ').append(entry.getKey()+" leafCount=" + pVal.parentLeaves);
+        if (pVal.childCounts != null) {
+          buf.append(' ').append(Arrays.toString(pVal.childCounts));
+        }
+      }
+      return buf.toString();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java
new file mode 100644
index 0000000..165c418
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.prefix;
+
+import com.spatial4j.core.shape.Point;
+import org.apache.lucene.spatial.prefix.tree.Cell;
+import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
+import org.apache.lucene.spatial.util.ShapeFieldCacheProvider;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Implementation of {@link ShapeFieldCacheProvider} designed for {@link PrefixTreeStrategy}s that index points
+ * (AND ONLY POINTS!).
+ *
+ * @lucene.internal
+ */
+public class PointPrefixTreeFieldCacheProvider extends ShapeFieldCacheProvider<Point> {
+
+  private final SpatialPrefixTree grid;
+  private Cell scanCell;//re-used in readShape to save GC
+
+  public PointPrefixTreeFieldCacheProvider(SpatialPrefixTree grid, String shapeField, int defaultSize) {
+    super( shapeField, defaultSize );
+    this.grid = grid;
+  }
+
+  @Override
+  protected Point readShape(BytesRef term) {
+    scanCell = grid.readCell(term, scanCell);
+    if (scanCell.getLevel() == grid.getMaxLevels())
+      return scanCell.getShape().getCenter();
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeFacetCounter.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeFacetCounter.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeFacetCounter.java
new file mode 100644
index 0000000..173c30e
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeFacetCounter.java
@@ -0,0 +1,201 @@
+/*
+ * 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.IndexReaderContext;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.spatial.prefix.tree.Cell;
+import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
+import org.apache.lucene.util.Bits;
+
+/**
+ * Computes facets on cells for {@link org.apache.lucene.spatial.prefix.PrefixTreeStrategy}.
+ * <p>
+ * <em>NOTE:</em> If for a given document and a given field using
+ * {@link org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy}
+ * multiple values are indexed (i.e. multi-valued) and at least one of them is a non-point, then there is a possibility
+ * of double-counting the document in the facet results.  Since each shape is independently turned into grid cells at
+ * a resolution chosen by the shape's size, it's possible they will be indexed at different resolutions.  This means
+ * the document could be present in BOTH the postings for a cell in both its prefix and leaf variants.  To avoid this,
+ * use a single valued field with a {@link com.spatial4j.core.shape.ShapeCollection} (or WKT equivalent).  Or
+ * calculate a suitable level/distErr to index both and call
+ * {@link org.apache.lucene.spatial.prefix.PrefixTreeStrategy#createIndexableFields(com.spatial4j.core.shape.Shape, int)}
+ * with the same value for all shapes for a given document/field.
+ *
+ * @lucene.experimental
+ */
+public class PrefixTreeFacetCounter {
+
+  /** A callback/visitor of facet counts. */
+  public static abstract class FacetVisitor {
+    /** Called at the start of the segment, if there is indexed data. */
+    public void startOfSegment() {}
+
+    /** Called for cells with a leaf, or cells at the target facet level.  {@code count} is greater than zero.
+     * When an ancestor cell is given with non-zero count, the count can be considered to be added to all cells
+     * below. You won't necessarily get a cell at level {@code facetLevel} if the indexed data is courser (bigger).
+     */
+    public abstract void visit(Cell cell, int count);
+  }
+
+  private PrefixTreeFacetCounter() {
+  }
+
+  /**
+   * Computes facets using a callback/visitor style design, allowing flexibility for the caller to determine what to do
+   * with each underlying count.
+   * @param strategy the prefix tree strategy (contains the field reference, grid, max levels)
+   * @param context the IndexReader's context
+   * @param topAcceptDocs a Bits to limit counted docs. If null, live docs are counted.
+   * @param queryShape the shape to limit the range of facet counts to
+   * @param facetLevel the maximum depth (detail) of faceted cells
+   * @param facetVisitor the visitor/callback to receive the counts
+   */
+  public static void compute(PrefixTreeStrategy strategy, IndexReaderContext context, Bits topAcceptDocs,
+                             Shape queryShape, int facetLevel, FacetVisitor facetVisitor)
+      throws IOException {
+    //We collect per-leaf
+    for (final LeafReaderContext leafCtx : context.leaves()) {
+      //determine leaf acceptDocs Bits
+      Bits leafAcceptDocs;
+      if (topAcceptDocs == null) {
+        leafAcceptDocs = leafCtx.reader().getLiveDocs();//filter deleted
+      } else {
+        leafAcceptDocs = new Bits() {
+          @Override
+          public boolean get(int index) {
+            return topAcceptDocs.get(leafCtx.docBase + index);
+          }
+
+          @Override
+          public int length() {
+            return leafCtx.reader().maxDoc();
+          }
+        };
+      }
+
+      compute(strategy, leafCtx, leafAcceptDocs, queryShape, facetLevel, facetVisitor);
+    }
+  }
+
+  /** Lower-level per-leaf segment method. */
+  public static void compute(final PrefixTreeStrategy strategy, final LeafReaderContext context, final Bits acceptDocs,
+                             final Shape queryShape, final int facetLevel, final FacetVisitor facetVisitor)
+      throws IOException {
+    if (acceptDocs != null && acceptDocs.length() != context.reader().maxDoc()) {
+      throw new IllegalArgumentException(
+          "acceptDocs bits length " + acceptDocs.length() +" != leaf maxdoc " + context.reader().maxDoc());
+    }
+    final SpatialPrefixTree tree = strategy.getGrid();
+
+    //scanLevel is an optimization knob of AbstractVisitingPrefixTreeFilter. It's unlikely
+    // another scanLevel would be much faster and it tends to be a risky knob (can help a little, can hurt a ton).
+    // TODO use RPT's configured scan level?  Do we know better here?  Hard to say.
+    final int scanLevel = tree.getMaxLevels();
+    //AbstractVisitingPrefixTreeFilter is a Lucene Filter.  We don't need a filter; we use it for its great prefix-tree
+    // traversal code.  TODO consider refactoring if/when it makes sense (more use cases than this)
+    new AbstractVisitingPrefixTreeQuery(queryShape, strategy.getFieldName(), tree, facetLevel, scanLevel) {
+
+      @Override
+      public String toString(String field) {
+        return "anonPrefixTreeQuery";//un-used
+      }
+
+      @Override
+      public DocIdSet getDocIdSet(LeafReaderContext contexts) throws IOException {
+        assert facetLevel == super.detailLevel;//same thing, FYI. (constant)
+
+        return new VisitorTemplate(context) {
+
+          @Override
+          protected void start() throws IOException {
+            facetVisitor.startOfSegment();
+          }
+
+          @Override
+          protected DocIdSet finish() throws IOException {
+            return null;//unused;
+          }
+
+          @Override
+          protected boolean visitPrefix(Cell cell) throws IOException {
+            // At facetLevel...
+            if (cell.getLevel() == facetLevel) {
+              // Count docs
+              visitLeaf(cell);//we're not a leaf but we treat it as such at facet level
+              return false;//don't descend further; this is enough detail
+            }
+
+            // We optimize for discriminating filters (reflected in acceptDocs) and short-circuit if no
+            // matching docs. We could do this at all levels or never but the closer we get to the facet level, the
+            // higher the probability this is worthwhile. We do when docFreq == 1 because it's a cheap check, especially
+            // due to "pulsing" in the codec.
+            //TODO this opt should move to VisitorTemplate (which contains an optimization TODO to this effect)
+            if (cell.getLevel() == facetLevel - 1 || termsEnum.docFreq() == 1) {
+              if (!hasDocsAtThisTerm()) {
+                return false;
+              }
+            }
+            return true;
+          }
+
+          @Override
+          protected void visitLeaf(Cell cell) throws IOException {
+            final int count = countDocsAtThisTerm();
+            if (count > 0) {
+              facetVisitor.visit(cell, count);
+            }
+          }
+
+          private int countDocsAtThisTerm() throws IOException {
+            if (acceptDocs == null) {
+              return termsEnum.docFreq();
+            }
+            int count = 0;
+            postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
+            while (postingsEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
+              if (acceptDocs.get(postingsEnum.docID()) == false) {
+                continue;
+              }
+              count++;
+            }
+            return count;
+          }
+
+          private boolean hasDocsAtThisTerm() throws IOException {
+            if (acceptDocs == null) {
+              return true;
+            }
+            postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
+            int nextDoc = postingsEnum.nextDoc();
+            while (nextDoc != DocIdSetIterator.NO_MORE_DOCS && acceptDocs.get(nextDoc) == false) {
+              nextDoc = postingsEnum.nextDoc();
+            }
+            return nextDoc != DocIdSetIterator.NO_MORE_DOCS;
+          }
+
+        }.getDocIdSet();
+      }
+    }.getDocIdSet(context);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
new file mode 100644
index 0000000..608879b
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
@@ -0,0 +1,208 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import com.spatial4j.core.shape.Point;
+import com.spatial4j.core.shape.Shape;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.IndexReaderContext;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.spatial.SpatialStrategy;
+import org.apache.lucene.spatial.prefix.tree.Cell;
+import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
+import org.apache.lucene.spatial.query.SpatialArgs;
+import org.apache.lucene.spatial.util.ShapeFieldCacheDistanceValueSource;
+import org.apache.lucene.util.Bits;
+
+/**
+ * An abstract SpatialStrategy based on {@link SpatialPrefixTree}. The two
+ * subclasses are {@link RecursivePrefixTreeStrategy} and {@link
+ * TermQueryPrefixTreeStrategy}.  This strategy is most effective as a fast
+ * approximate spatial search filter.
+ * <p>
+ * <b>Characteristics:</b>
+ * <br>
+ * <ul>
+ * <li>Can index any shape; however only {@link RecursivePrefixTreeStrategy}
+ * can effectively search non-point shapes.</li>
+ * <li>Can index a variable number of shapes per field value. This strategy
+ * can do it via multiple calls to {@link #createIndexableFields(com.spatial4j.core.shape.Shape)}
+ * for a document or by giving it some sort of Shape aggregate (e.g. JTS
+ * WKT MultiPoint).  The shape's boundary is approximated to a grid precision.
+ * </li>
+ * <li>Can query with any shape.  The shape's boundary is approximated to a grid
+ * precision.</li>
+ * <li>Only {@link org.apache.lucene.spatial.query.SpatialOperation#Intersects}
+ * is supported.  If only points are indexed then this is effectively equivalent
+ * to IsWithin.</li>
+ * <li>The strategy supports {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point,double)}
+ * even for multi-valued data, so long as the indexed data is all points; the
+ * behavior is undefined otherwise.  However, <em>it will likely be removed in
+ * the future</em> in lieu of using another strategy with a more scalable
+ * implementation.  Use of this call is the only
+ * circumstance in which a cache is used.  The cache is simple but as such
+ * it doesn't scale to large numbers of points nor is it real-time-search
+ * friendly.</li>
+ * </ul>
+ * <p>
+ * <b>Implementation:</b>
+ * <p>
+ * The {@link SpatialPrefixTree} does most of the work, for example returning
+ * a list of terms representing grids of various sizes for a supplied shape.
+ * An important
+ * configuration item is {@link #setDistErrPct(double)} which balances
+ * shape precision against scalability.  See those javadocs.
+ *
+ * @lucene.experimental
+ */
+public abstract class PrefixTreeStrategy extends SpatialStrategy {
+  protected final SpatialPrefixTree grid;
+  private final Map<String, PointPrefixTreeFieldCacheProvider> provider = new ConcurrentHashMap<>();
+  protected int defaultFieldValuesArrayLen = 2;
+  protected double distErrPct = SpatialArgs.DEFAULT_DISTERRPCT;// [ 0 TO 0.5 ]
+  protected boolean pointsOnly = false;//if true, there are no leaves
+
+  public PrefixTreeStrategy(SpatialPrefixTree grid, String fieldName) {
+    super(grid.getSpatialContext(), fieldName);
+    this.grid = grid;
+  }
+
+  public SpatialPrefixTree getGrid() {
+    return grid;
+  }
+
+  /**
+   * A memory hint used by {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point)}
+   * for how big the initial size of each Document's array should be. The
+   * default is 2.  Set this to slightly more than the default expected number
+   * of points per document.
+   */
+  public void setDefaultFieldValuesArrayLen(int defaultFieldValuesArrayLen) {
+    this.defaultFieldValuesArrayLen = defaultFieldValuesArrayLen;
+  }
+
+  public double getDistErrPct() {
+    return distErrPct;
+  }
+
+  /**
+   * The default measure of shape precision affecting shapes at index and query
+   * times. Points don't use this as they are always indexed at the configured
+   * maximum precision ({@link org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree#getMaxLevels()});
+   * this applies to all other shapes. Specific shapes at index and query time
+   * can use something different than this default value.  If you don't set a
+   * default then the default is {@link SpatialArgs#DEFAULT_DISTERRPCT} --
+   * 2.5%.
+   *
+   * @see org.apache.lucene.spatial.query.SpatialArgs#getDistErrPct()
+   */
+  public void setDistErrPct(double distErrPct) {
+    this.distErrPct = distErrPct;
+  }
+
+  public boolean isPointsOnly() {
+    return pointsOnly;
+  }
+
+  /** True if only indexed points shall be supported. There are no "leafs" in such a case, except those
+   * at maximum precision. */
+  public void setPointsOnly(boolean pointsOnly) {
+    this.pointsOnly = pointsOnly;
+  }
+
+  @Override
+  public Field[] createIndexableFields(Shape shape) {
+    double distErr = SpatialArgs.calcDistanceFromErrPct(shape, distErrPct, ctx);
+    return createIndexableFields(shape, distErr);
+  }
+
+  /**
+   * Turns {@link SpatialPrefixTree#getTreeCellIterator(Shape, int)} into a
+   * {@link org.apache.lucene.analysis.TokenStream}.
+   */
+  public Field[] createIndexableFields(Shape shape, double distErr) {
+    int detailLevel = grid.getLevelForDistance(distErr);
+    return createIndexableFields(shape, detailLevel);
+  }
+
+  public Field[] createIndexableFields(Shape shape, int detailLevel) {
+    //TODO re-use TokenStream LUCENE-5776: Subclass Field, put cell iterator there, override tokenStream()
+    Iterator<Cell> cells = createCellIteratorToIndex(shape, detailLevel, null);
+    CellToBytesRefIterator cellToBytesRefIterator = newCellToBytesRefIterator();
+    cellToBytesRefIterator.reset(cells);
+    BytesRefIteratorTokenStream tokenStream = new BytesRefIteratorTokenStream();
+    tokenStream.setBytesRefIterator(cellToBytesRefIterator);
+    Field field = new Field(getFieldName(), tokenStream, FIELD_TYPE);
+    return new Field[]{field};
+  }
+
+  protected CellToBytesRefIterator newCellToBytesRefIterator() {
+    //subclasses could return one that never emits leaves, or does both, or who knows.
+    return new CellToBytesRefIterator();
+  }
+
+  protected Iterator<Cell> createCellIteratorToIndex(Shape shape, int detailLevel, Iterator<Cell> reuse) {
+    if (pointsOnly && !(shape instanceof Point)) {
+      throw new IllegalArgumentException("pointsOnly is true yet a " + shape.getClass() + " is given for indexing");
+    }
+    return grid.getTreeCellIterator(shape, detailLevel);//TODO should take a re-use iterator
+  }
+
+  /* Indexed, tokenized, not stored. */
+  public static final FieldType FIELD_TYPE = new FieldType();
+
+  static {
+    FIELD_TYPE.setTokenized(true);
+    FIELD_TYPE.setOmitNorms(true);
+    FIELD_TYPE.setIndexOptions(IndexOptions.DOCS);
+    FIELD_TYPE.freeze();
+  }
+
+  @Override
+  public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
+    PointPrefixTreeFieldCacheProvider p = provider.get( getFieldName() );
+    if( p == null ) {
+      synchronized (this) {//double checked locking idiom is okay since provider is threadsafe
+        p = provider.get( getFieldName() );
+        if (p == null) {
+          p = new PointPrefixTreeFieldCacheProvider(grid, getFieldName(), defaultFieldValuesArrayLen);
+          provider.put(getFieldName(),p);
+        }
+      }
+    }
+
+    return new ShapeFieldCacheDistanceValueSource(ctx, p, queryPoint, multiplier);
+  }
+
+  /**
+   * Computes spatial facets in two dimensions as a grid of numbers.  The data is often visualized as a so-called
+   * "heatmap".
+   *
+   * @see HeatmapFacetCounter#calcFacets(PrefixTreeStrategy, IndexReaderContext, Bits, Shape, int, int)
+   */
+  public HeatmapFacetCounter.Heatmap calcFacets(IndexReaderContext context, Bits topAcceptDocs,
+                                   Shape inputShape, final int facetLevel, int maxCells) throws IOException {
+    return HeatmapFacetCounter.calcFacets(this, context, topAcceptDocs, inputShape, facetLevel, maxCells);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java
new file mode 100644
index 0000000..68b0449
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java
@@ -0,0 +1,192 @@
+/*
+ * 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.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import com.spatial4j.core.shape.Point;
+import com.spatial4j.core.shape.Shape;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.spatial.prefix.tree.Cell;
+import org.apache.lucene.spatial.prefix.tree.CellIterator;
+import org.apache.lucene.spatial.prefix.tree.LegacyCell;
+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;
+
+/**
+ * A {@link PrefixTreeStrategy} which uses {@link AbstractVisitingPrefixTreeQuery}.
+ * This strategy has support for searching non-point shapes (note: not tested).
+ * Even a query shape with distErrPct=0 (fully precise to the grid) should have
+ * good performance for typical data, unless there is a lot of indexed data
+ * coincident with the shape's edge.
+ *
+ * @lucene.experimental
+ */
+public class RecursivePrefixTreeStrategy extends PrefixTreeStrategy {
+  /* Future potential optimizations:
+
+    Each shape.relate(otherShape) result could be cached since much of the same relations will be invoked when
+    multiple segments are involved. Do this for "complex" shapes, not cheap ones, and don't cache when disjoint to
+    bbox because it's a cheap calc. This is one advantage TermQueryPrefixTreeStrategy has over RPT.
+
+   */
+
+  protected int prefixGridScanLevel;
+
+  //Formerly known as simplifyIndexedCells. Eventually will be removed. Only compatible with RPT
+  // and a LegacyPrefixTree.
+  protected boolean pruneLeafyBranches = true;
+
+  protected boolean multiOverlappingIndexedShapes = true;
+
+  public RecursivePrefixTreeStrategy(SpatialPrefixTree grid, String fieldName) {
+    super(grid, fieldName);
+    prefixGridScanLevel = grid.getMaxLevels() - 4;//TODO this default constant is dependent on the prefix grid size
+  }
+
+  public int getPrefixGridScanLevel() {
+    return prefixGridScanLevel;
+  }
+
+  /**
+   * Sets the grid level [1-maxLevels] at which indexed terms are scanned brute-force
+   * instead of by grid decomposition.  By default this is maxLevels - 4.  The
+   * final level, maxLevels, is always scanned.
+   *
+   * @param prefixGridScanLevel 1 to maxLevels
+   */
+  public void setPrefixGridScanLevel(int prefixGridScanLevel) {
+    //TODO if negative then subtract from maxlevels
+    this.prefixGridScanLevel = prefixGridScanLevel;
+  }
+
+  public boolean isMultiOverlappingIndexedShapes() {
+    return multiOverlappingIndexedShapes;
+  }
+
+  /** See {@link ContainsPrefixTreeQuery#multiOverlappingIndexedShapes}. */
+  public void setMultiOverlappingIndexedShapes(boolean multiOverlappingIndexedShapes) {
+    this.multiOverlappingIndexedShapes = multiOverlappingIndexedShapes;
+  }
+
+  public boolean isPruneLeafyBranches() {
+    return pruneLeafyBranches;
+  }
+
+  /**
+   * An optional hint affecting non-point shapes: it will
+   * prune away a complete set sibling leaves to their parent (recursively), resulting in ~20-50%
+   * fewer indexed cells, and consequently that much less disk and that much faster indexing.
+   * So if it's a quad tree and all 4 sub-cells are there marked as a leaf, then they will be
+   * removed (pruned) and the parent is marked as a leaf instead.  This occurs recursively on up.  Unfortunately, the
+   * current implementation will buffer all cells to do this, so consider disabling for high precision (low distErrPct)
+   * shapes. (default=true)
+   */
+  public void setPruneLeafyBranches(boolean pruneLeafyBranches) {
+    this.pruneLeafyBranches = pruneLeafyBranches;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder str = new StringBuilder(getClass().getSimpleName()).append('(');
+    str.append("SPG:(").append(grid.toString()).append(')');
+    if (pointsOnly)
+      str.append(",pointsOnly");
+    if (pruneLeafyBranches)
+      str.append(",pruneLeafyBranches");
+    if (prefixGridScanLevel != grid.getMaxLevels() - 4)
+      str.append(",prefixGridScanLevel:").append(""+prefixGridScanLevel);
+    if (!multiOverlappingIndexedShapes)
+      str.append(",!multiOverlappingIndexedShapes");
+    return str.append(')').toString();
+  }
+
+  @Override
+  protected Iterator<Cell> createCellIteratorToIndex(Shape shape, int detailLevel, Iterator<Cell> reuse) {
+    if (shape instanceof Point || !pruneLeafyBranches)
+      return super.createCellIteratorToIndex(shape, detailLevel, reuse);
+
+    List<Cell> cells = new ArrayList<>(4096);
+    recursiveTraverseAndPrune(grid.getWorldCell(), shape, detailLevel, cells);
+    return cells.iterator();
+  }
+
+  /** Returns true if cell was added as a leaf. If it wasn't it recursively descends. */
+  private boolean recursiveTraverseAndPrune(Cell cell, Shape shape, int detailLevel, List<Cell> result) {
+    // Important: this logic assumes Cells don't share anything with other cells when
+    // calling cell.getNextLevelCells(). This is only true for LegacyCell.
+    if (!(cell instanceof LegacyCell))
+      throw new IllegalStateException("pruneLeafyBranches must be disabled for use with grid "+grid);
+
+    if (cell.getLevel() == detailLevel) {
+      cell.setLeaf();//FYI might already be a leaf
+    }
+    if (cell.isLeaf()) {
+      result.add(cell);
+      return true;
+    }
+    if (cell.getLevel() != 0)
+      result.add(cell);
+
+    int leaves = 0;
+    CellIterator subCells = cell.getNextLevelCells(shape);
+    while (subCells.hasNext()) {
+      Cell subCell = subCells.next();
+      if (recursiveTraverseAndPrune(subCell, shape, detailLevel, result))
+        leaves++;
+    }
+    //can we prune?
+    if (leaves == ((LegacyCell)cell).getSubCellsSize() && cell.getLevel() != 0) {
+      //Optimization: substitute the parent as a leaf instead of adding all
+      // children as leaves
+
+      //remove the leaves
+      do {
+        result.remove(result.size() - 1);//remove last
+      } while (--leaves > 0);
+      //add cell as the leaf
+      cell.setLeaf();
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public Query makeQuery(SpatialArgs args) {
+    final SpatialOperation op = args.getOperation();
+
+    Shape shape = args.getShape();
+    int detailLevel = grid.getLevelForDistance(args.resolveDistErr(ctx, distErrPct));
+
+    if (op == SpatialOperation.Intersects) {
+      return new IntersectsPrefixTreeQuery(
+          shape, getFieldName(), grid, detailLevel, prefixGridScanLevel);
+    } else if (op == SpatialOperation.IsWithin) {
+      return new WithinPrefixTreeQuery(
+          shape, getFieldName(), grid, detailLevel, prefixGridScanLevel,
+          -1);//-1 flag is slower but ensures correct results
+    } else if (op == SpatialOperation.Contains) {
+      return new ContainsPrefixTreeQuery(shape, getFieldName(), grid, detailLevel,
+          multiOverlappingIndexedShapes);
+    }
+    throw new UnsupportedSpatialOperation(op);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
new file mode 100644
index 0000000..a74786b
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
@@ -0,0 +1,111 @@
+/*
+ * 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.ArrayList;
+import java.util.List;
+
+import com.spatial4j.core.shape.Point;
+import com.spatial4j.core.shape.Shape;
+import org.apache.lucene.queries.TermsQuery;
+import org.apache.lucene.search.Query;
+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.spatial.query.SpatialArgs;
+import org.apache.lucene.spatial.query.SpatialOperation;
+import org.apache.lucene.spatial.query.UnsupportedSpatialOperation;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+
+/**
+ * A basic implementation of {@link PrefixTreeStrategy} using a large
+ * {@link TermsQuery} of all the cells from
+ * {@link SpatialPrefixTree#getTreeCellIterator(com.spatial4j.core.shape.Shape, int)}.
+ * It only supports the search of indexed Point shapes.
+ * <p>
+ * The precision of query shapes (distErrPct) is an important factor in using
+ * this Strategy. If the precision is too precise then it will result in many
+ * terms which will amount to a slower query.
+ *
+ * @lucene.experimental
+ */
+public class TermQueryPrefixTreeStrategy extends PrefixTreeStrategy {
+
+  protected boolean simplifyIndexedCells = false;
+
+  public TermQueryPrefixTreeStrategy(SpatialPrefixTree grid, String fieldName) {
+    super(grid, fieldName);
+  }
+
+  @Override
+  protected CellToBytesRefIterator newCellToBytesRefIterator() {
+    //Ensure we don't have leaves, as this strategy doesn't handle them.
+    return new CellToBytesRefIterator() {
+      @Override
+      public BytesRef next() {
+        if (!cellIter.hasNext()) {
+          return null;
+        }
+        return cellIter.next().getTokenBytesNoLeaf(bytesRef);
+      }
+    };
+  }
+
+  @Override
+  public Query makeQuery(SpatialArgs args) {
+    final SpatialOperation op = args.getOperation();
+    if (op != SpatialOperation.Intersects)
+      throw new UnsupportedSpatialOperation(op);
+
+    Shape shape = args.getShape();
+    int detailLevel = grid.getLevelForDistance(args.resolveDistErr(ctx, distErrPct));
+
+    //--get a List of BytesRef for each term we want (no parents, no leaf bytes))
+    final int GUESS_NUM_TERMS;
+    if (shape instanceof Point)
+      GUESS_NUM_TERMS = detailLevel;//perfect guess
+    else
+      GUESS_NUM_TERMS = 4096;//should this be a method on SpatialPrefixTree?
+
+    BytesRefBuilder masterBytes = new BytesRefBuilder();//shared byte array for all terms
+    List<BytesRef> terms = new ArrayList<>(GUESS_NUM_TERMS);
+
+    CellIterator cells = grid.getTreeCellIterator(shape, detailLevel);
+    while (cells.hasNext()) {
+      Cell cell = cells.next();
+      if (!cell.isLeaf())
+        continue;
+      BytesRef term = cell.getTokenBytesNoLeaf(null);//null because we want a new BytesRef
+      //We copy out the bytes because it may be re-used across the iteration. This also gives us the opportunity
+      // to use one contiguous block of memory for the bytes of all terms we need.
+      masterBytes.grow(masterBytes.length() + term.length);
+      masterBytes.append(term);
+      term.bytes = null;//don't need; will reset later
+      term.offset = masterBytes.length() - term.length;
+      terms.add(term);
+    }
+    //doing this now because if we did earlier, it's possible the bytes needed to grow()
+    for (BytesRef byteRef : terms) {
+      byteRef.bytes = masterBytes.bytes();
+    }
+    //unfortunately TermsQuery will needlessly sort & dedupe
+    //TODO an automatonQuery might be faster?
+    return new TermsQuery(getFieldName(), terms);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/WithinPrefixTreeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/WithinPrefixTreeQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/WithinPrefixTreeQuery.java
new file mode 100644
index 0000000..cf0d11b
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/WithinPrefixTreeQuery.java
@@ -0,0 +1,232 @@
+/*
+ * 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.context.SpatialContext;
+import com.spatial4j.core.distance.DistanceUtils;
+import com.spatial4j.core.shape.Circle;
+import com.spatial4j.core.shape.Point;
+import com.spatial4j.core.shape.Rectangle;
+import com.spatial4j.core.shape.Shape;
+import com.spatial4j.core.shape.SpatialRelation;
+import org.apache.lucene.index.LeafReaderContext;
+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.BitDocIdSet;
+import org.apache.lucene.util.FixedBitSet;
+
+/**
+ * Finds docs where its indexed shape is {@link org.apache.lucene.spatial.query.SpatialOperation#IsWithin
+ * WITHIN} the query shape.  It works by looking at cells outside of the query
+ * shape to ensure documents there are excluded. By default, it will
+ * examine all cells, and it's fairly slow.  If you know that the indexed shapes
+ * are never comprised of multiple disjoint parts (which also means it is not multi-valued),
+ * then you can pass {@code SpatialPrefixTree.getDistanceForLevel(maxLevels)} as
+ * the {@code queryBuffer} constructor parameter to minimally look this distance
+ * beyond the query shape's edge.  Even if the indexed shapes are sometimes
+ * comprised of multiple disjoint parts, you might want to use this option with
+ * a large buffer as a faster approximation with minimal false-positives.
+ *
+ * @lucene.experimental
+ */
+public class WithinPrefixTreeQuery extends AbstractVisitingPrefixTreeQuery {
+  //TODO LUCENE-4869: implement faster algorithm based on filtering out false-positives of a
+  //  minimal query buffer by looking in a DocValues cache holding a representative
+  //  point of each disjoint component of a document's shape(s).
+
+  //TODO Could the recursion in allCellsIntersectQuery() be eliminated when non-fuzzy or other
+  //  circumstances?
+
+  private final Shape bufferedQueryShape;//if null then the whole world
+
+  /**
+   * See {@link AbstractVisitingPrefixTreeQuery#AbstractVisitingPrefixTreeQuery(com.spatial4j.core.shape.Shape, String, org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree, int, int)}.
+   * {@code queryBuffer} is the (minimum) distance beyond the query shape edge
+   * where non-matching documents are looked for so they can be excluded. If
+   * -1 is used then the whole world is examined (a good default for correctness).
+   */
+  public WithinPrefixTreeQuery(Shape queryShape, String fieldName, SpatialPrefixTree grid,
+                               int detailLevel, int prefixGridScanLevel,
+                               double queryBuffer) {
+    super(queryShape, fieldName, grid, detailLevel, prefixGridScanLevel);
+    this.bufferedQueryShape = queryBuffer == -1 ? null : bufferShape(queryShape, queryBuffer);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (!super.equals(o)) return false;//checks getClass == o.getClass & instanceof
+
+    WithinPrefixTreeQuery that = (WithinPrefixTreeQuery) o;
+
+    if (bufferedQueryShape != null ? !bufferedQueryShape.equals(that.bufferedQueryShape) : that.bufferedQueryShape != null)
+      return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    result = 31 * result + (bufferedQueryShape != null ? bufferedQueryShape.hashCode() : 0);
+    return result;
+  }
+
+  @Override
+  public String toString(String field) {
+    return getClass().getSimpleName() + "(" +
+             "fieldName=" + fieldName + "," +
+             "queryShape=" + queryShape + "," +
+             "detailLevel=" + detailLevel + "," +
+             "prefixGridScanLevel=" + prefixGridScanLevel +
+           ")";
+  }
+
+  /** Returns a new shape that is larger than shape by at distErr.
+   */
+  //TODO move this generic code elsewhere?  Spatial4j?
+  protected Shape bufferShape(Shape shape, double distErr) {
+    if (distErr <= 0)
+      throw new IllegalArgumentException("distErr must be > 0");
+    SpatialContext ctx = grid.getSpatialContext();
+    if (shape instanceof Point) {
+      return ctx.makeCircle((Point)shape, distErr);
+    } else if (shape instanceof Circle) {
+      Circle circle = (Circle) shape;
+      double newDist = circle.getRadius() + distErr;
+      if (ctx.isGeo() && newDist > 180)
+        newDist = 180;
+      return ctx.makeCircle(circle.getCenter(), newDist);
+    } else {
+      Rectangle bbox = shape.getBoundingBox();
+      double newMinX = bbox.getMinX() - distErr;
+      double newMaxX = bbox.getMaxX() + distErr;
+      double newMinY = bbox.getMinY() - distErr;
+      double newMaxY = bbox.getMaxY() + distErr;
+      if (ctx.isGeo()) {
+        if (newMinY < -90)
+          newMinY = -90;
+        if (newMaxY > 90)
+          newMaxY = 90;
+        if (newMinY == -90 || newMaxY == 90 || bbox.getWidth() + 2*distErr > 360) {
+          newMinX = -180;
+          newMaxX = 180;
+        } else {
+          newMinX = DistanceUtils.normLonDEG(newMinX);
+          newMaxX = DistanceUtils.normLonDEG(newMaxX);
+        }
+      } else {
+        //restrict to world bounds
+        newMinX = Math.max(newMinX, ctx.getWorldBounds().getMinX());
+        newMaxX = Math.min(newMaxX, ctx.getWorldBounds().getMaxX());
+        newMinY = Math.max(newMinY, ctx.getWorldBounds().getMinY());
+        newMaxY = Math.min(newMaxY, ctx.getWorldBounds().getMaxY());
+      }
+      return ctx.makeRectangle(newMinX, newMaxX, newMinY, newMaxY);
+    }
+  }
+
+
+  @Override
+  protected DocIdSet getDocIdSet(LeafReaderContext context) throws IOException {
+    return new VisitorTemplate(context) {
+      private FixedBitSet inside;
+      private FixedBitSet outside;
+
+      @Override
+      protected void start() {
+        inside = new FixedBitSet(maxDoc);
+        outside = new FixedBitSet(maxDoc);
+      }
+
+      @Override
+      protected DocIdSet finish() {
+        inside.andNot(outside);
+        return new BitDocIdSet(inside);
+      }
+
+      @Override
+      protected CellIterator findSubCellsToVisit(Cell cell) {
+        //use buffered query shape instead of orig.  Works with null too.
+        return cell.getNextLevelCells(bufferedQueryShape);
+      }
+
+      @Override
+      protected boolean visitPrefix(Cell cell) throws IOException {
+        //cell.relate is based on the bufferedQueryShape; we need to examine what
+        // the relation is against the queryShape
+        SpatialRelation visitRelation = cell.getShape().relate(queryShape);
+        if (cell.getLevel() == detailLevel) {
+          collectDocs(visitRelation.intersects() ? inside : outside);
+          return false;
+        } else if (visitRelation == SpatialRelation.WITHIN) {
+          collectDocs(inside);
+          return false;
+        } else if (visitRelation == SpatialRelation.DISJOINT) {
+          collectDocs(outside);
+          return false;
+        }
+        return true;
+      }
+
+      @Override
+      protected void visitLeaf(Cell cell) throws IOException {
+        if (allCellsIntersectQuery(cell))
+          collectDocs(inside);
+        else
+          collectDocs(outside);
+      }
+
+      /** Returns true if the provided cell, and all its sub-cells down to
+       * detailLevel all intersect the queryShape.
+       */
+      private boolean allCellsIntersectQuery(Cell cell) {
+        SpatialRelation relate = cell.getShape().relate(queryShape);
+        if (cell.getLevel() == detailLevel)
+          return relate.intersects();
+        if (relate == SpatialRelation.WITHIN)
+          return true;
+        if (relate == SpatialRelation.DISJOINT)
+          return false;
+        // Note: Generating all these cells just to determine intersection is not ideal.
+        // The real solution is LUCENE-4869.
+        CellIterator subCells = cell.getNextLevelCells(null);
+        while (subCells.hasNext()) {
+          Cell subCell = subCells.next();
+          if (!allCellsIntersectQuery(subCell))//recursion
+            return false;
+        }
+        return true;
+      }
+
+      @Override
+      protected void visitScanned(Cell cell) throws IOException {
+        visitLeaf(cell);//collects as we want, even if not a leaf
+//        if (cell.isLeaf()) {
+//          visitLeaf(cell);
+//        } else {
+//          visitPrefix(cell);
+//        }
+      }
+
+    }.getDocIdSet();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89db4950/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/package-info.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/package-info.java
new file mode 100644
index 0000000..b35f8ee
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/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.
+ */
+
+/**
+ * Prefix Tree Strategy.
+ */
+package org.apache.lucene.spatial.prefix;