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/05 18:03:26 UTC

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/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/50a2f754/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/50a2f754/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/50a2f754/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/50a2f754/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/50a2f754/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/50a2f754/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..3796116
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/WithinPrefixTreeQuery.java
@@ -0,0 +1,233 @@
+/*
+ * 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.Bits;
+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/50a2f754/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..2749cfd
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/package-info.java
@@ -0,0 +1,20 @@
+/*
+ * 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;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java
new file mode 100644
index 0000000..fe3846d
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.prefix.tree;
+
+import com.spatial4j.core.shape.Shape;
+import com.spatial4j.core.shape.SpatialRelation;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Represents a grid cell. Cell instances are generally very transient and may be re-used
+ * internally.  To get an instance, you could start with {@link SpatialPrefixTree#getWorldCell()}.
+ * And from there you could either traverse down the tree with {@link #getNextLevelCells(com.spatial4j.core.shape.Shape)},
+ * or you could read an indexed term via {@link SpatialPrefixTree#readCell(org.apache.lucene.util.BytesRef,Cell)}.
+ * When a cell is read from a term, it is comprised of just the base bytes plus optionally a leaf flag.
+ *
+ * @lucene.experimental
+ */
+public interface Cell {
+
+//  If we bring this back; perhaps do so as a method that un-shares its internal state: void unshare();
+//  /** Resets the state of this cell such that it is identical to {@code source}. This can be used for
+//   * cloning a cell to have a safe copy, and it also might be used to position this cell
+//   * before calling {@link #readCell(org.apache.lucene.util.BytesRef)} in a loop if you know the first term
+//   * is going to be close to some other cell, thereby saving some computations. */
+//  void copyFrom(Cell source);
+
+  /** Gets the relationship this cell has with the shape from which it was filtered from, assuming it came from a
+   * {@link CellIterator}. Arguably it belongs there but it's very convenient here. */
+  SpatialRelation getShapeRel();
+
+  /** See {@link #getShapeRel()}.
+   * @lucene.internal */
+  void setShapeRel(SpatialRelation rel);
+
+  /**
+   * Some cells are flagged as leaves, which are indexed as such. A leaf cell is either within some
+   * shape or it both intersects and the cell is at an accuracy threshold such that no smaller cells
+   * for the shape will be represented.
+   */
+  boolean isLeaf();
+
+  /** Set this cell to be a leaf. Warning: never call on a cell
+   * initialized to reference the same bytes from termsEnum, which should be treated as immutable.
+   * Note: not supported at level 0.
+   * @lucene.internal */
+  void setLeaf();
+
+  /**
+   * Returns the bytes for this cell, with a leaf byte <em>if this is a leaf cell</em>.
+   * The result param is used to save object allocation, though its bytes aren't used.
+   * @param result where the result goes, or null to create new
+   */
+  BytesRef getTokenBytesWithLeaf(BytesRef result);
+
+  /**
+   * Returns the bytes for this cell, without a leaf set. The bytes should sort before
+   * {@link #getTokenBytesWithLeaf(org.apache.lucene.util.BytesRef)}.
+   * The result param is used to save object allocation, though its bytes aren't used.
+   * @param result where the result goes, or null to create new
+   */
+  BytesRef getTokenBytesNoLeaf(BytesRef result);
+
+  /** Level 0 is the world (and has no parent), from then on a higher level means a smaller
+   * cell than the level before it.
+   */
+  int getLevel();
+
+  /**
+   * Gets the cells at the next grid cell level underneath this one, optionally filtered by
+   * {@code shapeFilter}. The returned cells should have {@link #getShapeRel()} set to
+   * their relation with {@code shapeFilter}.  In addition, for non-points {@link #isLeaf()}
+   * must be true when that relation is WITHIN.
+   * <p>
+   * IMPORTANT: Cells returned from this iterator can be shared, as well as the bytes.
+   * <p>
+   * Precondition: Never called when getLevel() == maxLevel.
+   *
+   * @param shapeFilter an optional filter for the returned cells.
+   * @return A set of cells (no dups), sorted. Not Modifiable.
+   */
+  CellIterator getNextLevelCells(Shape shapeFilter);
+
+  /** Gets the shape for this cell; typically a Rectangle. */
+  Shape getShape();
+
+  /**
+   * Returns if the target term is within/underneath this cell; not necessarily a direct
+   * descendant.
+   * @param c the term
+   */
+  boolean isPrefixOf(Cell c);
+
+  /** Equivalent to {@code this.getTokenBytesNoLeaf(null).compareTo(fromCell.getTokenBytesNoLeaf(null))}. */
+  int compareToNoLeaf(Cell fromCell);
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/CellIterator.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/CellIterator.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/CellIterator.java
new file mode 100644
index 0000000..1cef37a
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/CellIterator.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.prefix.tree;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+/**
+ * An Iterator of SpatialPrefixTree Cells. The order is always sorted without duplicates.
+ *
+ * @lucene.experimental
+ */
+public abstract class CellIterator implements Iterator<Cell> {
+
+  //note: nextCell or thisCell can be non-null but neither at the same time. That's
+  // because they might return the same instance when re-used!
+
+  protected Cell nextCell;//to be returned by next(), and null'ed after
+  protected Cell thisCell;//see next() & thisCell(). Should be cleared in hasNext().
+
+  /** Returns the cell last returned from {@link #next()}. It's cleared by hasNext(). */
+  public Cell thisCell() {
+    assert thisCell != null : "Only call thisCell() after next(), not hasNext()";
+    return thisCell;
+  }
+
+  // Arguably this belongs here and not on Cell
+  //public SpatialRelation getShapeRel()
+
+  /**
+   * Gets the next cell that is &gt;= {@code fromCell}, compared using non-leaf bytes. If it returns null then
+   * the iterator is exhausted.
+   */
+  public Cell nextFrom(Cell fromCell) {
+    while (true) {
+      if (!hasNext())
+        return null;
+      Cell c = next();//will update thisCell
+      if (c.compareToNoLeaf(fromCell) >= 0) {
+        return c;
+      }
+    }
+  }
+
+  /** This prevents sub-cells (those underneath the current cell) from being iterated to,
+   *  if applicable, otherwise a NO-OP. */
+  @Override
+  public void remove() {
+    assert thisCell != null;
+  }
+
+  @Override
+  public Cell next() {
+    if (nextCell == null) {
+      if (!hasNext())
+        throw new NoSuchElementException();
+    }
+    thisCell = nextCell;
+    nextCell = null;
+    return thisCell;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTree.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTree.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTree.java
new file mode 100644
index 0000000..13281f3
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTree.java
@@ -0,0 +1,444 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.prefix.tree;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.Locale;
+import java.util.TimeZone;
+
+import com.spatial4j.core.shape.Shape;
+
+/**
+ * A PrefixTree for date ranges in which the levels of the tree occur at natural periods of time (e.g. years,
+ * months, ...). You pass in {@link Calendar} objects with the desired fields set and the unspecified
+ * fields unset, which conveys the precision.  The implementation makes some optimization assumptions about a
+ * {@link java.util.GregorianCalendar}; others could probably be supported easily.
+ * <p>
+ * Warning: If you construct a Calendar and then get something from the object like a field (e.g. year) or
+ * milliseconds, then every field is fully set by side-effect. So after setting the fields, pass it to this
+ * API first.
+ * @lucene.experimental
+ */
+public class DateRangePrefixTree extends NumberRangePrefixTree {
+
+  /*
+    WARNING  java.util.Calendar is tricky to work with:
+    * If you "get" any field value, every field becomes "set". This can introduce a Heisenbug effect,
+        when in a debugger in some cases. Fortunately, Calendar.toString() doesn't apply.
+    * Beware Calendar underflow of the underlying long.  If you create a Calendar from LONG.MIN_VALUE, and clear
+     a field, it will underflow and appear close to LONG.MAX_VALUE (BC to AD).
+
+    There are no doubt other reasons but those two were hard fought lessons here.
+
+    TODO Improvements:
+    * Make max precision configurable (i.e. to SECOND).
+    * Make min & max year span configurable. Use that to remove pointless top levels of the SPT.
+        If year span is > 10k, then add 1k year level. If year span is > 10k of 1k levels, add 1M level.
+    * NumberRangePrefixTree: override getTreeCellIterator for optimized case where the shape isn't a date span; use
+      FilterCellIterator of the cell stack.
+
+  */
+
+  private static final TimeZone UTC = TimeZone.getTimeZone("UTC");
+  private static Calendar CAL_TMP;//template
+  static {
+    CAL_TMP = Calendar.getInstance(UTC, Locale.ROOT);
+    CAL_TMP.clear();
+  }
+
+  private static final Calendar MINCAL = (Calendar) CAL_TMP.clone();
+  private static final Calendar MAXCAL = (Calendar) CAL_TMP.clone();
+  static {
+    MINCAL.setTimeInMillis(Long.MIN_VALUE);
+    MAXCAL.setTimeInMillis(Long.MAX_VALUE);
+  }
+  //BC years are decreasing, remember.  Yet ActualMaximum is the numerically high value, ActualMinimum is 1.
+  private static final int BC_FIRSTYEAR = MINCAL.getActualMaximum(Calendar.YEAR);
+  private static final int BC_LASTYEAR = MINCAL.getActualMinimum(Calendar.YEAR);//1
+  private static final int BC_YEARS = BC_FIRSTYEAR - BC_LASTYEAR + 1;
+  private static final int AD_FIRSTYEAR = MAXCAL.getActualMinimum(Calendar.YEAR);//1
+  private static final int AD_LASTYEAR = MAXCAL.getActualMaximum(Calendar.YEAR);
+  private static final int AD_YEAR_BASE =  (((BC_YEARS-1) / 1000_000)+1) * 1000_000;
+  static { assert BC_LASTYEAR == 1 && AD_FIRSTYEAR == 1; }
+
+  //how many million years are there?
+  private static final int NUM_MYEARS = (AD_YEAR_BASE + AD_LASTYEAR) / 1000_000;
+
+  private static int calFieldLen(int field) {
+    return CAL_TMP.getMaximum(field) - CAL_TMP.getMinimum(field) + 1;
+  }
+
+  private static final int[] FIELD_BY_LEVEL = {
+      -1/*unused*/, -1, -1, Calendar.YEAR, Calendar.MONTH, Calendar.DAY_OF_MONTH,
+      Calendar.HOUR_OF_DAY, Calendar.MINUTE, Calendar.SECOND, Calendar.MILLISECOND};
+  private static final int yearLevel = 3;
+
+  public static final DateRangePrefixTree INSTANCE = new DateRangePrefixTree();
+
+  private final UnitNRShape minLV, maxLV;
+  private final UnitNRShape gregorianChangeDateLV;
+
+  protected DateRangePrefixTree() {
+    super(new int[]{//sublevels by level
+        NUM_MYEARS,
+        1000,//1 thousand thousand-years in a million years
+        1000,//1 thousand years in a thousand-year
+        calFieldLen(Calendar.MONTH),
+        calFieldLen(Calendar.DAY_OF_MONTH),
+        calFieldLen(Calendar.HOUR_OF_DAY),
+        calFieldLen(Calendar.MINUTE),
+        calFieldLen(Calendar.SECOND),
+        calFieldLen(Calendar.MILLISECOND),
+    });
+    maxLV = toShape((Calendar)MAXCAL.clone());
+    minLV = toShape((Calendar)MINCAL.clone());
+    if (MAXCAL instanceof GregorianCalendar) {
+      //TODO this should be a configurable param by passing a Calendar serving as a template.
+      GregorianCalendar gCal = (GregorianCalendar)MAXCAL;
+      gregorianChangeDateLV = toUnitShape(gCal.getGregorianChange());
+    } else {
+      gregorianChangeDateLV = null;
+    }
+  }
+
+  @Override
+  public int getNumSubCells(UnitNRShape lv) {
+    int cmp = comparePrefix(lv, maxLV);
+    assert cmp <= 0;
+    if (cmp == 0)//edge case (literally!)
+      return maxLV.getValAtLevel(lv.getLevel()+1);
+
+    // if using GregorianCalendar and we're after the "Gregorian change date" then we'll compute
+    //  the sub-cells ourselves more efficiently without the need to construct a Calendar.
+    cmp = gregorianChangeDateLV != null ? comparePrefix(lv, gregorianChangeDateLV) : -1;
+    //TODO consider also doing fast-path if field is <= hours even if before greg change date
+    if (cmp >= 0) {
+      int result = fastSubCells(lv);
+      assert result == slowSubCells(lv) : "fast/slow numSubCells inconsistency";
+      return result;
+    } else {
+      return slowSubCells(lv);
+    }
+  }
+
+  private int fastSubCells(UnitNRShape lv) {
+    if (lv.getLevel() == yearLevel+1) {//month
+      switch (lv.getValAtLevel(lv.getLevel())) {
+        case Calendar.SEPTEMBER:
+        case Calendar.APRIL:
+        case Calendar.JUNE:
+        case Calendar.NOVEMBER:
+          return 30;
+        case Calendar.FEBRUARY:
+          //get the year (negative numbers for BC)
+          int yearAdj = lv.getValAtLevel(1) * 1_000_000;
+          yearAdj += lv.getValAtLevel(2) * 1000;
+          yearAdj += lv.getValAtLevel(3);
+          int year = yearAdj - AD_YEAR_BASE;
+          if (year % 4 == 0 && !(year % 100 == 0 && year % 400 != 0) )//leap year
+            return 29;
+          else
+            return 28;
+        default:
+          return 31;
+      }
+    } else {//typical:
+      return super.getNumSubCells(lv);
+    }
+  }
+
+  private int slowSubCells(UnitNRShape lv) {
+    int field = FIELD_BY_LEVEL[lv.getLevel()+1];
+    //short-circuit optimization (GregorianCalendar assumptions)
+    if (field == -1 || field == Calendar.YEAR || field >= Calendar.HOUR_OF_DAY)//TODO make configurable
+      return super.getNumSubCells(lv);
+    Calendar cal = toCalendar(lv);//somewhat heavyweight op; ideally should be stored on UnitNRShape somehow
+    return cal.getActualMaximum(field) - cal.getActualMinimum(field) + 1;
+  }
+
+  /** Calendar utility method:
+   * Returns a new {@link Calendar} in UTC TimeZone, ROOT Locale, with all fields cleared. */
+  public Calendar newCal() {
+    return (Calendar) CAL_TMP.clone();
+  }
+
+  /** Calendar utility method:
+   * Returns the spatial prefix tree level for the corresponding {@link java.util.Calendar} field, such as
+   * {@link java.util.Calendar#YEAR}.  If there's no match, the next greatest level is returned as a negative value.
+   */
+  public int getTreeLevelForCalendarField(int calField) {
+    for (int i = yearLevel; i < FIELD_BY_LEVEL.length; i++) {
+      if (FIELD_BY_LEVEL[i] == calField) {
+        return i;
+      } else if (FIELD_BY_LEVEL[i] > calField) {
+        return -1 * i;
+      }
+    }
+    throw new IllegalArgumentException("Bad calendar field?: " + calField);
+  }
+
+  /** Calendar utility method:
+   * Gets the Calendar field code of the last field that is set prior to an unset field. It only
+   * examines fields relevant to the prefix tree. If no fields are set, it returns -1. */
+  public int getCalPrecisionField(Calendar cal) {
+    int lastField = -1;
+    for (int level = yearLevel; level < FIELD_BY_LEVEL.length; level++) {
+      int field = FIELD_BY_LEVEL[level];
+      if (!cal.isSet(field))
+        break;
+      lastField = field;
+    }
+    return lastField;
+  }
+
+  /** Calendar utility method:
+   * Calls {@link Calendar#clear(int)} for every field after {@code field}. Beware of Calendar underflow. */
+  public void clearFieldsAfter(Calendar cal, int field) {
+    if (field == -1) {
+      cal.clear();
+      return;
+    }
+    int assertEra = -1;
+    assert (assertEra = (((Calendar)cal.clone()).get(Calendar.ERA))) >= 0;//a trick to only get this if assert enabled
+    for (int f = field+1; f < Calendar.FIELD_COUNT; f++) {
+      cal.clear(f);
+    }
+    assert ((Calendar)cal.clone()).get(Calendar.ERA) == assertEra : "Calendar underflow";
+  }
+
+  /** Converts {@code value} from a {@link Calendar} or {@link Date} to a {@link Shape}. Other arguments
+   * result in a {@link java.lang.IllegalArgumentException}.
+   */
+  @Override
+  public UnitNRShape toUnitShape(Object value) {
+    if (value instanceof Calendar) {
+      return toShape((Calendar) value);
+    } else if (value instanceof Date) {
+      Calendar cal = newCal();
+      cal.setTime((Date)value);
+      return toShape(cal);
+    }
+    throw new IllegalArgumentException("Expecting Calendar or Date but got: "+value.getClass());
+  }
+
+  /** Converts the Calendar into a Shape.
+   * The isSet() state of the Calendar is re-instated when done. */
+  public UnitNRShape toShape(Calendar cal) {
+    // Convert a Calendar into a stack of cell numbers
+    final int calPrecField = getCalPrecisionField(cal);//must call first; getters set all fields
+    try {
+      int[] valStack = new int[maxLevels];//starts at level 1, not 0
+      int len = 0;
+      if (calPrecField >= Calendar.YEAR) {//year or better precision
+        int year = cal.get(Calendar.YEAR);
+        int yearAdj = cal.get(Calendar.ERA) == 0 ? AD_YEAR_BASE - (year - 1) : AD_YEAR_BASE + year;
+
+        valStack[len++] = yearAdj / 1000_000;
+        yearAdj -= valStack[len-1] * 1000_000;
+        valStack[len++] = yearAdj / 1000;
+        yearAdj -= valStack[len-1] * 1000;
+        valStack[len++] = yearAdj;
+        for (int level = yearLevel+1; level < FIELD_BY_LEVEL.length; level++) {
+          int field = FIELD_BY_LEVEL[level];
+          if (field > calPrecField)
+            break;
+          valStack[len++] = cal.get(field) - cal.getActualMinimum(field);
+        }
+      }
+
+      return toShape(valStack, len);
+    } finally {
+      clearFieldsAfter(cal, calPrecField);//restore precision state modified by get()
+    }
+  }
+
+  /** Calls {@link #toCalendar(org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape)}. */
+  @Override
+  public Object toObject(UnitNRShape shape) {
+    return toCalendar(shape);
+  }
+
+  /** Converts the {@link org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape} shape to a
+   * corresponding Calendar that is cleared below its level. */
+  public Calendar toCalendar(UnitNRShape lv) {
+    if (lv.getLevel() == 0)
+      return newCal();
+    if (comparePrefix(lv, minLV) <= 0) {//shouldn't typically happen; sometimes in a debugger
+      return (Calendar) MINCAL.clone();//full precision; truncation would cause underflow
+    }
+    assert comparePrefix(lv, maxLV) <= 0;
+    Calendar cal = newCal();
+
+    int yearAdj = lv.getValAtLevel(1) * 1_000_000;
+    if (lv.getLevel() > 1) {
+      yearAdj += lv.getValAtLevel(2) * 1000;
+      if (lv.getLevel() > 2) {
+        yearAdj += lv.getValAtLevel(3);
+      }
+    }
+    if (yearAdj > AD_YEAR_BASE) {
+      cal.set(Calendar.ERA, 1);
+      cal.set(Calendar.YEAR, yearAdj - AD_YEAR_BASE);//setting the year resets the era
+    } else {
+      cal.set(Calendar.ERA, 0);//we assert this "sticks" at the end
+      cal.set(Calendar.YEAR, (AD_YEAR_BASE - yearAdj) + 1);
+    }
+    for (int level = yearLevel+1; level <= lv.getLevel(); level++) {
+      int field = FIELD_BY_LEVEL[level];
+      cal.set(field, lv.getValAtLevel(level) + cal.getActualMinimum(field));
+    }
+    assert yearAdj > AD_YEAR_BASE || ((Calendar)cal.clone()).get(Calendar.ERA) == 0 : "ERA / YEAR underflow";
+    return cal;
+  }
+
+  @Override
+  protected String toString(UnitNRShape lv) {
+    return toString(toCalendar(lv));
+  }
+
+  /** Calendar utility method:
+   * Formats the calendar to ISO-8601 format, to include proper BC handling (1BC is "0000", 2BC is "-0001", etc.);
+   * and WITHOUT a trailing 'Z'.
+   * A fully cleared calendar will yield the string "*".
+   * The isSet() state of the Calendar is re-instated when done. */
+   @SuppressWarnings("fallthrough")
+  public String toString(Calendar cal) {
+    final int calPrecField = getCalPrecisionField(cal);//must call first; getters set all fields
+    if (calPrecField == -1)
+      return "*";
+    try {
+      //TODO not fully optimized; but it's at least not used in 'search'.
+      //TODO maybe borrow code from Solr DateUtil (put in Lucene util somewhere), and have it reference this back?
+      String pattern = "yyyy-MM-dd'T'HH:mm:ss.SSS";
+      int ptnLen = 0;
+      switch (calPrecField) {//switch fall-through is deliberate
+        case Calendar.MILLISECOND: ptnLen += 4;
+        case Calendar.SECOND: ptnLen += 3;
+        case Calendar.MINUTE: ptnLen += 3;
+        case Calendar.HOUR_OF_DAY: ptnLen += 5;
+        case Calendar.DAY_OF_MONTH: ptnLen += 3;
+        case Calendar.MONTH: ptnLen += 3;
+        case Calendar.YEAR: ptnLen += 4;
+        break;
+        default: throw new IllegalStateException(""+calPrecField);
+      }
+      pattern = pattern.substring(0, ptnLen);
+      SimpleDateFormat format = new SimpleDateFormat(pattern, Locale.ROOT);
+      format.setTimeZone(cal.getTimeZone());
+      if (cal.get(Calendar.ERA) == 0) {//BC
+        //SDF doesn't do this properly according to ISO-8601
+        // Example: 1BC == "0000" (actually 0 AD), 2BC == "-0001", 3BC == "-0002", ...
+        final int yearOrig = cal.get(Calendar.YEAR);
+        cal.set(Calendar.YEAR, yearOrig-1);
+        String str;
+        try {
+          str = format.format(cal.getTime());
+        } finally {
+          //reset to what it was
+          cal.set(Calendar.ERA, 0);//necessary!
+          cal.set(Calendar.YEAR, yearOrig);
+        }
+        if (yearOrig > 1)
+          return "-" + str;
+        else
+          return "0000" + str.substring(4);
+      }
+      return format.format(cal.getTime());
+    } finally {
+      clearFieldsAfter(cal, calPrecField);//restore precision state modified by get()
+    }
+  }
+
+  @Override
+  protected UnitNRShape parseUnitShape(String str) throws ParseException {
+    return toShape(parseCalendar(str));
+  }
+
+  /** Calendar utility method:
+   * The reverse of {@link #toString(java.util.Calendar)}. It will only set the fields found, leaving
+   * the remainder in an un-set state. A leading '-' or '+' is optional (positive assumed), and a
+   * trailing 'Z' is also optional.
+   * @param str not null and not empty
+   * @return not null
+   */
+  public Calendar parseCalendar(String str) throws ParseException {
+    // example: +2014-10-23T21:22:33.159Z
+    if (str == null || str.isEmpty())
+      throw new IllegalArgumentException("str is null or blank");
+    Calendar cal = newCal();
+    if (str.equals("*"))
+      return cal;
+    int offset = 0;//a pointer
+    try {
+      //year & era:
+      int lastOffset = str.charAt(str.length()-1) == 'Z' ? str.length() - 1 : str.length();
+      int hyphenIdx = str.indexOf('-', 1);//look past possible leading hyphen
+      if (hyphenIdx < 0)
+        hyphenIdx = lastOffset;
+      int year = Integer.parseInt(str.substring(offset, hyphenIdx));
+      cal.set(Calendar.ERA, year <= 0 ? 0 : 1);
+      cal.set(Calendar.YEAR, year <= 0 ? -1*year + 1 : year);
+      offset = hyphenIdx + 1;
+      if (lastOffset < offset)
+        return cal;
+
+      //NOTE: We aren't validating separator chars, and we unintentionally accept leading +/-.
+      // The str.substring()'s hopefully get optimized to be stack-allocated.
+
+      //month:
+      cal.set(Calendar.MONTH, Integer.parseInt(str.substring(offset, offset+2)) - 1);//starts at 0
+      offset += 3;
+      if (lastOffset < offset)
+        return cal;
+      //day:
+      cal.set(Calendar.DAY_OF_MONTH, Integer.parseInt(str.substring(offset, offset+2)));
+      offset += 3;
+      if (lastOffset < offset)
+        return cal;
+      //hour:
+      cal.set(Calendar.HOUR_OF_DAY, Integer.parseInt(str.substring(offset, offset+2)));
+      offset += 3;
+      if (lastOffset < offset)
+        return cal;
+      //minute:
+      cal.set(Calendar.MINUTE, Integer.parseInt(str.substring(offset, offset+2)));
+      offset += 3;
+      if (lastOffset < offset)
+        return cal;
+      //second:
+      cal.set(Calendar.SECOND, Integer.parseInt(str.substring(offset, offset+2)));
+      offset += 3;
+      if (lastOffset < offset)
+        return cal;
+      //ms:
+      cal.set(Calendar.MILLISECOND, Integer.parseInt(str.substring(offset, offset+3)));
+      offset += 3;//last one, move to next char
+      if (lastOffset == offset)
+        return cal;
+    } catch (Exception e) {
+      ParseException pe = new ParseException("Improperly formatted date: "+str, offset);
+      pe.initCause(e);
+      throw pe;
+    }
+    throw new ParseException("Improperly formatted date: "+str, offset);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/FilterCellIterator.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/FilterCellIterator.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/FilterCellIterator.java
new file mode 100644
index 0000000..e4f50e0
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/FilterCellIterator.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.prefix.tree;
+
+import com.spatial4j.core.shape.Shape;
+import com.spatial4j.core.shape.SpatialRelation;
+
+import java.util.Iterator;
+
+/**
+ * A filtering iterator of Cells. Those not matching the provided shape (disjoint) are
+ * skipped. If {@code shapeFilter} is null then all cells are returned.
+ *
+ * @lucene.internal
+ */
+class FilterCellIterator extends CellIterator {
+  final Iterator<Cell> baseIter;
+  final Shape shapeFilter;
+
+  FilterCellIterator(Iterator<Cell> baseIter, Shape shapeFilter) {
+    this.baseIter = baseIter;
+    this.shapeFilter = shapeFilter;
+  }
+
+  @Override
+  public boolean hasNext() {
+    thisCell = null;
+    if (nextCell != null)//calling hasNext twice in a row
+      return true;
+    while (baseIter.hasNext()) {
+      nextCell = baseIter.next();
+      if (shapeFilter == null) {
+        return true;
+      } else {
+        SpatialRelation rel = nextCell.getShape().relate(shapeFilter);
+        if (rel.intersects()) {
+          nextCell.setShapeRel(rel);
+          if (rel == SpatialRelation.WITHIN)
+            nextCell.setLeaf();
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+}