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:11:09 UTC

[28/34] lucene-solr git commit: LUCENE-7015: 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/5b028349/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java
new file mode 100644
index 0000000..fa4e987
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java
@@ -0,0 +1,162 @@
+/*
+ * 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.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import com.spatial4j.core.context.SpatialContext;
+import com.spatial4j.core.io.GeohashUtils;
+import com.spatial4j.core.shape.Point;
+import com.spatial4j.core.shape.Rectangle;
+import com.spatial4j.core.shape.Shape;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A {@link SpatialPrefixTree} based on
+ * <a href="http://en.wikipedia.org/wiki/Geohash">Geohashes</a>.
+ * Uses {@link GeohashUtils} to do all the geohash work.
+ *
+ * @lucene.experimental
+ */
+public class GeohashPrefixTree extends LegacyPrefixTree {
+
+  /**
+   * Factory for creating {@link GeohashPrefixTree} instances with useful defaults
+   */
+  public static class Factory extends SpatialPrefixTreeFactory {
+
+    @Override
+    protected int getLevelForDistance(double degrees) {
+      GeohashPrefixTree grid = new GeohashPrefixTree(ctx, GeohashPrefixTree.getMaxLevelsPossible());
+      return grid.getLevelForDistance(degrees);
+    }
+
+    @Override
+    protected SpatialPrefixTree newSPT() {
+      return new GeohashPrefixTree(ctx,
+          maxLevels != null ? maxLevels : GeohashPrefixTree.getMaxLevelsPossible());
+    }
+  }
+
+  public GeohashPrefixTree(SpatialContext ctx, int maxLevels) {
+    super(ctx, maxLevels);
+    Rectangle bounds = ctx.getWorldBounds();
+    if (bounds.getMinX() != -180)
+      throw new IllegalArgumentException("Geohash only supports lat-lon world bounds. Got "+bounds);
+    int MAXP = getMaxLevelsPossible();
+    if (maxLevels <= 0 || maxLevels > MAXP)
+      throw new IllegalArgumentException("maxLevels must be [1-"+MAXP+"] but got "+ maxLevels);
+  }
+
+  /** Any more than this and there's no point (double lat and lon are the same). */
+  public static int getMaxLevelsPossible() {
+    return GeohashUtils.MAX_PRECISION;
+  }
+
+  @Override
+  public Cell getWorldCell() {
+    return new GhCell(BytesRef.EMPTY_BYTES, 0, 0);
+  }
+
+  @Override
+  public int getLevelForDistance(double dist) {
+    if (dist == 0)
+      return maxLevels;//short circuit
+    final int level = GeohashUtils.lookupHashLenForWidthHeight(dist, dist);
+    return Math.max(Math.min(level, maxLevels), 1);
+  }
+
+  @Override
+  protected Cell getCell(Point p, int level) {
+    return new GhCell(GeohashUtils.encodeLatLon(p.getY(), p.getX(), level));//args are lat,lon (y,x)
+  }
+
+  private static byte[] stringToBytesPlus1(String token) {
+    //copy ASCII token to byte array with one extra spot for eventual LEAF_BYTE if needed
+    byte[] bytes = new byte[token.length() + 1];
+    for (int i = 0; i < token.length(); i++) {
+      bytes[i] = (byte) token.charAt(i);
+    }
+    return bytes;
+  }
+
+  private class GhCell extends LegacyCell {
+
+    private String geohash;//cache; never has leaf byte, simply a geohash
+
+    GhCell(String geohash) {
+      super(stringToBytesPlus1(geohash), 0, geohash.length());
+      this.geohash = geohash;
+      if (isLeaf() && getLevel() < getMaxLevels())//we don't have a leaf byte at max levels (an opt)
+        this.geohash = geohash.substring(0, geohash.length() - 1);
+    }
+
+    GhCell(byte[] bytes, int off, int len) {
+      super(bytes, off, len);
+    }
+
+    @Override
+    protected GeohashPrefixTree getGrid() { return GeohashPrefixTree.this; }
+
+    @Override
+    protected int getMaxLevels() { return maxLevels; }
+
+    @Override
+    protected void readCell(BytesRef bytesRef) {
+      super.readCell(bytesRef);
+      geohash = null;
+    }
+
+    @Override
+    public Collection<Cell> getSubCells() {
+      String[] hashes = GeohashUtils.getSubGeohashes(getGeohash());//sorted
+      List<Cell> cells = new ArrayList<>(hashes.length);
+      for (String hash : hashes) {
+        cells.add(new GhCell(hash));
+      }
+      return cells;
+    }
+
+    @Override
+    public int getSubCellsSize() {
+      return 32;//8x4
+    }
+
+    @Override
+    protected GhCell getSubCell(Point p) {
+      return (GhCell) getGrid().getCell(p, getLevel() + 1);//not performant!
+    }
+
+    @Override
+    public Shape getShape() {
+      if (shape == null) {
+        shape = GeohashUtils.decodeBoundary(getGeohash(), getGrid().getSpatialContext());
+      }
+      return shape;
+    }
+
+    private String getGeohash() {
+      if (geohash == null)
+        geohash = getTokenBytesNoLeaf(null).utf8ToString();
+      return geohash;
+    }
+
+  }//class GhCell
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5b028349/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java
new file mode 100644
index 0000000..27c56a7
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java
@@ -0,0 +1,242 @@
+/*
+ * 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.Collection;
+
+import com.spatial4j.core.shape.Point;
+import com.spatial4j.core.shape.Shape;
+import com.spatial4j.core.shape.SpatialRelation;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.StringHelper;
+
+/** The base for the original two SPT's: Geohash and Quad. Don't subclass this for new SPTs.
+ * @lucene.internal */
+//public for RPT pruneLeafyBranches code
+public abstract class LegacyCell implements Cell {
+
+  // Important: A LegacyCell doesn't share state for getNextLevelCells(), and
+  //  LegacySpatialPrefixTree assumes this in its simplify tree logic.
+
+  private static final byte LEAF_BYTE = '+';//NOTE: must sort before letters & numbers
+
+  //Arguably we could simply use a BytesRef, using an extra Object.
+  protected byte[] bytes;//generally bigger to potentially hold a leaf
+  protected int b_off;
+  protected int b_len;//doesn't reflect leaf; same as getLevel()
+
+  protected boolean isLeaf;
+
+  /**
+   * When set via getSubCells(filter), it is the relationship between this cell
+   * and the given shape filter. Doesn't participate in shape equality.
+   */
+  protected SpatialRelation shapeRel;
+
+  protected Shape shape;//cached
+
+  /** Warning: Refers to the same bytes (no copy). If {@link #setLeaf()} is subsequently called then it
+   * may modify bytes. */
+  protected LegacyCell(byte[] bytes, int off, int len) {
+    this.bytes = bytes;
+    this.b_off = off;
+    this.b_len = len;
+    readLeafAdjust();
+  }
+
+  protected void readCell(BytesRef bytes) {
+    shapeRel = null;
+    shape = null;
+    this.bytes = bytes.bytes;
+    this.b_off = bytes.offset;
+    this.b_len = (short) bytes.length;
+    readLeafAdjust();
+  }
+
+  protected void readLeafAdjust() {
+    isLeaf = (b_len > 0 && bytes[b_off + b_len - 1] == LEAF_BYTE);
+    if (isLeaf)
+      b_len--;
+    if (getLevel() == getMaxLevels())
+      isLeaf = true;
+  }
+
+  protected abstract SpatialPrefixTree getGrid();
+
+  protected abstract int getMaxLevels();
+
+  @Override
+  public SpatialRelation getShapeRel() {
+    return shapeRel;
+  }
+
+  @Override
+  public void setShapeRel(SpatialRelation rel) {
+    this.shapeRel = rel;
+  }
+
+  @Override
+  public boolean isLeaf() {
+    return isLeaf;
+  }
+
+  @Override
+  public void setLeaf() {
+    isLeaf = true;
+  }
+
+  @Override
+  public BytesRef getTokenBytesWithLeaf(BytesRef result) {
+    result = getTokenBytesNoLeaf(result);
+    if (!isLeaf || getLevel() == getMaxLevels())
+      return result;
+    if (result.bytes.length < result.offset + result.length + 1) {
+      assert false : "Not supposed to happen; performance bug";
+      byte[] copy = new byte[result.length + 1];
+      System.arraycopy(result.bytes, result.offset, copy, 0, result.length - 1);
+      result.bytes = copy;
+      result.offset = 0;
+    }
+    result.bytes[result.offset + result.length++] = LEAF_BYTE;
+    return result;
+  }
+
+  @Override
+  public BytesRef getTokenBytesNoLeaf(BytesRef result) {
+    if (result == null)
+      return new BytesRef(bytes, b_off, b_len);
+    result.bytes = bytes;
+    result.offset = b_off;
+    result.length = b_len;
+    return result;
+  }
+
+  @Override
+  public int getLevel() {
+    return b_len;
+  }
+
+  @Override
+  public CellIterator getNextLevelCells(Shape shapeFilter) {
+    assert getLevel() < getGrid().getMaxLevels();
+    if (shapeFilter instanceof Point) {
+      LegacyCell cell = getSubCell((Point) shapeFilter);
+      cell.shapeRel = SpatialRelation.CONTAINS;
+      return new SingletonCellIterator(cell);
+    } else {
+      return new FilterCellIterator(getSubCells().iterator(), shapeFilter);
+    }
+  }
+
+  /**
+   * Performant implementations are expected to implement this efficiently by
+   * considering the current cell's boundary.
+   * <p>
+   * Precondition: Never called when getLevel() == maxLevel.
+   * Precondition: this.getShape().relate(p) != DISJOINT.
+   */
+  protected abstract LegacyCell getSubCell(Point p);
+
+  /**
+   * Gets the cells at the next grid cell level that covers this cell.
+   * Precondition: Never called when getLevel() == maxLevel.
+   *
+   * @return A set of cells (no dups), sorted, modifiable, not empty, not null.
+   */
+  protected abstract Collection<Cell> getSubCells();
+
+  /**
+   * {@link #getSubCells()}.size() -- usually a constant. Should be &gt;=2
+   */
+  public abstract int getSubCellsSize();
+
+  @Override
+  public boolean isPrefixOf(Cell c) {
+    //Note: this only works when each level uses a whole number of bytes.
+    LegacyCell cell = (LegacyCell)c;
+    boolean result = sliceEquals(cell.bytes, cell.b_off, cell.b_len, bytes, b_off, b_len);
+    assert result == StringHelper.startsWith(c.getTokenBytesNoLeaf(null), getTokenBytesNoLeaf(null));
+    return result;
+  }
+
+  /** Copied from {@link org.apache.lucene.util.StringHelper#startsWith(BytesRef, BytesRef)}
+   *  which calls this. This is to avoid creating a BytesRef.  */
+  private static boolean sliceEquals(byte[] sliceToTest_bytes, int sliceToTest_offset, int sliceToTest_length,
+                                     byte[] other_bytes, int other_offset, int other_length) {
+    if (sliceToTest_length < other_length) {
+      return false;
+    }
+    int i = sliceToTest_offset;
+    int j = other_offset;
+    final int k = other_offset + other_length;
+
+    while (j < k) {
+      if (sliceToTest_bytes[i++] != other_bytes[j++]) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  @Override
+  public int compareToNoLeaf(Cell fromCell) {
+    LegacyCell b = (LegacyCell) fromCell;
+    return compare(bytes, b_off, b_len, b.bytes, b.b_off, b.b_len);
+  }
+
+  /** Copied from {@link BytesRef#compareTo(BytesRef)}.
+   * This is to avoid creating a BytesRef. */
+  protected static int compare(byte[] aBytes, int aUpto, int a_length, byte[] bBytes, int bUpto, int b_length) {
+    final int aStop = aUpto + Math.min(a_length, b_length);
+    while(aUpto < aStop) {
+      int aByte = aBytes[aUpto++] & 0xff;
+      int bByte = bBytes[bUpto++] & 0xff;
+
+      int diff = aByte - bByte;
+      if (diff != 0) {
+        return diff;
+      }
+    }
+
+    // One is a prefix of the other, or, they are equal:
+    return a_length - b_length;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    //this method isn't "normally" called; just in asserts/tests
+    if (obj instanceof Cell) {
+      Cell cell = (Cell) obj;
+      return getTokenBytesWithLeaf(null).equals(cell.getTokenBytesWithLeaf(null));
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return getTokenBytesWithLeaf(null).hashCode();
+  }
+
+  @Override
+  public String toString() {
+    //this method isn't "normally" called; just in asserts/tests
+    return getTokenBytesWithLeaf(null).utf8ToString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5b028349/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/LegacyPrefixTree.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/LegacyPrefixTree.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/LegacyPrefixTree.java
new file mode 100644
index 0000000..672c2fe
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/LegacyPrefixTree.java
@@ -0,0 +1,84 @@
+/*
+ * 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.Arrays;
+
+import com.spatial4j.core.context.SpatialContext;
+import com.spatial4j.core.shape.Point;
+import com.spatial4j.core.shape.Rectangle;
+import com.spatial4j.core.shape.Shape;
+import org.apache.lucene.util.BytesRef;
+
+/** The base for the original two SPT's: Geohash and Quad. Don't subclass this for new SPTs.
+ * @lucene.internal */
+abstract class LegacyPrefixTree extends SpatialPrefixTree {
+  public LegacyPrefixTree(SpatialContext ctx, int maxLevels) {
+    super(ctx, maxLevels);
+  }
+
+  public double getDistanceForLevel(int level) {
+    if (level < 1 || level > getMaxLevels())
+      throw new IllegalArgumentException("Level must be in 1 to maxLevels range");
+    //TODO cache for each level
+    Cell cell = getCell(ctx.getWorldBounds().getCenter(), level);
+    Rectangle bbox = cell.getShape().getBoundingBox();
+    double width = bbox.getWidth();
+    double height = bbox.getHeight();
+    //Use standard cartesian hypotenuse. For geospatial, this answer is larger
+    // than the correct one but it's okay to over-estimate.
+    return Math.sqrt(width * width + height * height);
+  }
+
+  /**
+   * Returns the cell containing point {@code p} at the specified {@code level}.
+   */
+  protected abstract Cell getCell(Point p, int level);
+
+  @Override
+  public Cell readCell(BytesRef term, Cell scratch) {
+    LegacyCell cell = (LegacyCell) scratch;
+    if (cell == null)
+      cell = (LegacyCell) getWorldCell();
+    cell.readCell(term);
+    return cell;
+  }
+
+  @Override
+  public CellIterator getTreeCellIterator(Shape shape, int detailLevel) {
+    if (!(shape instanceof Point))
+      return super.getTreeCellIterator(shape, detailLevel);
+
+    //This specialization is here because the legacy implementations don't have a fast implementation of
+    // cell.getSubCells(point). It's fastest here to encode the full bytes for detailLevel, and create
+    // subcells from the bytesRef in a loop. This avoids an O(N^2) encode, and we have O(N) instead.
+
+    Cell cell = getCell((Point) shape, detailLevel);
+    assert cell instanceof LegacyCell;
+    BytesRef fullBytes = cell.getTokenBytesNoLeaf(null);
+    //fill in reverse order to be sorted
+    Cell[] cells = new Cell[detailLevel];
+    for (int i = 1; i < detailLevel; i++) {
+      fullBytes.length = i;
+      Cell parentCell = readCell(fullBytes, null);
+      cells[i-1] = parentCell;
+    }
+    cells[detailLevel-1] = cell;
+    return new FilterCellIterator(Arrays.asList(cells).iterator(), null);//null filter
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5b028349/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/NumberRangePrefixTree.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/NumberRangePrefixTree.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/NumberRangePrefixTree.java
new file mode 100644
index 0000000..40e80bc
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/NumberRangePrefixTree.java
@@ -0,0 +1,989 @@
+/*
+ * 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 com.spatial4j.core.context.SpatialContext;
+import com.spatial4j.core.context.SpatialContextFactory;
+import com.spatial4j.core.shape.Point;
+import com.spatial4j.core.shape.Rectangle;
+import com.spatial4j.core.shape.Shape;
+import com.spatial4j.core.shape.SpatialRelation;
+import com.spatial4j.core.shape.impl.RectangleImpl;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.StringHelper;
+
+/**
+ * A SpatialPrefixTree for single-dimensional numbers and number ranges of fixed precision values (not floating point).
+ * Despite its name, the indexed values (and queries) need not actually be ranges, they can be unit instance/values.
+ * <p>
+ * Why might you use this instead of Lucene's built-in integer/long support?  Here are some reasons with features based
+ * on code in this class, <em>or are possible based on this class but require a subclass to fully realize it</em>.
+ * <ul>
+ *   <li>Index ranges, not just unit instances. This is especially useful when the requirement calls for a
+ *   multi-valued range.</li>
+ *   <li>Instead of a fixed "precisionStep", this prefixTree can have a customizable number of child values for any
+ *   prefix (up to 32768). This allows exact alignment of the prefix-tree with typical/expected values, which
+ *   results in better performance.  For example in a Date implementation, every month can get its own dedicated prefix,
+ *   every day, etc., even though months vary in duration.</li>
+ *   <li>Arbitrary precision, like {@link java.math.BigDecimal}.</li>
+ *   <li>Standard Lucene integer/long indexing always indexes the full precision of those data types but this one
+ *   is customizable.</li>
+ * </ul>
+ *
+ * Unlike "normal" spatial components in this module, this special-purpose one only works with {@link Shape}s
+ * created by the methods on this class, not from any {@link com.spatial4j.core.context.SpatialContext}.
+ *
+ * @see org.apache.lucene.spatial.prefix.NumberRangePrefixTreeStrategy
+ * @see <a href="https://issues.apache.org/jira/browse/LUCENE-5648">LUCENE-5648</a>
+ * @lucene.experimental
+ */
+public abstract class NumberRangePrefixTree extends SpatialPrefixTree {
+
+  //
+  //    Dummy SpatialContext
+  //
+
+  private static final SpatialContext DUMMY_CTX;
+  static {
+    SpatialContextFactory factory = new SpatialContextFactory();
+    factory.geo = false;
+    factory.worldBounds = new RectangleImpl(Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, 0L, 0L, null);
+    DUMMY_CTX = factory.newSpatialContext();
+  }
+
+  /** Base interface for {@link Shape}s this prefix tree supports. It extends {@link Shape} (Spatial4j) for compatibility
+   * with the spatial API even though it doesn't intermix with conventional 2D shapes.
+   * @lucene.experimental
+   */
+  public static interface NRShape extends Shape, Cloneable {
+    /** The result should be parseable by {@link #parseShape(String)}. */
+    abstract String toString();
+
+    /** Returns this shape rounded to the target level. If we are already more course than the level then the shape is
+     * simply returned.  The result may refer to internal state of the argument so you may want to clone it.
+     */
+    public NRShape roundToLevel(int targetLevel);
+  }
+
+  //
+  //  Factory / Conversions / parsing relating to NRShapes
+  //
+
+  /** Converts the value to a unit shape. Doesn't parse strings; see {@link #parseShape(String)} for
+   * that. This is the reverse of {@link #toObject(org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape)}. */
+  public abstract UnitNRShape toUnitShape(Object value);
+
+  /** Returns a shape that represents the continuous range between {@code start} and {@code end}. It will
+   * be normalized, and so sometimes a {@link org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape}
+   * will be returned, other times a
+   * {@link org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.SpanUnitsNRShape} will be.
+   *
+   * @throws IllegalArgumentException if the arguments are in the wrong order, or if either contains the other (yet they
+   * aren't equal).
+   */
+  public NRShape toRangeShape(UnitNRShape startUnit, UnitNRShape endUnit) {
+    //note: this normalization/optimization process is actually REQUIRED based on assumptions elsewhere.
+    //Normalize start & end
+    startUnit = startUnit.getShapeAtLevel(truncateStartVals(startUnit, 0)); // chops off trailing min-vals (zeroes)
+    endUnit = endUnit.getShapeAtLevel(truncateEndVals(endUnit, 0)); // chops off trailing max-vals
+    //Optimize to just start or end if it's equivalent, e.g. April to April 1st is April 1st.
+    int cmp = comparePrefix(startUnit, endUnit);
+    if (cmp > 0) {
+      throw new IllegalArgumentException("Wrong order: "+ startUnit +" TO "+ endUnit);
+    }
+    if (cmp == 0) {//one is a prefix of the other
+      if (startUnit.getLevel() == endUnit.getLevel()) {
+        //same
+        return startUnit;
+      } else if (endUnit.getLevel() > startUnit.getLevel()) {
+        // e.g. April to April 1st
+        if (truncateStartVals(endUnit, startUnit.getLevel()) == startUnit.getLevel()) {
+          return endUnit;
+        }
+      } else {//minLV level > maxLV level
+        // e.g. April 30 to April
+        if (truncateEndVals(startUnit, endUnit.getLevel()) == endUnit.getLevel()) {
+          return startUnit;
+        }
+      }
+    }
+    return new SpanUnitsNRShape(startUnit, endUnit);
+  }
+
+  /** From lv.getLevel on up, it returns the first Level seen with val != 0. It doesn't check past endLevel. */
+  private int truncateStartVals(UnitNRShape lv, int endLevel) {
+    for (int level = lv.getLevel(); level > endLevel; level--) {
+      if (lv.getValAtLevel(level) != 0)
+        return level;
+    }
+    return endLevel;
+  }
+
+  private int truncateEndVals(UnitNRShape lv, int endLevel) {
+    for (int level = lv.getLevel(); level > endLevel; level--) {
+      int max = getNumSubCells(lv.getShapeAtLevel(level - 1)) - 1;
+      if (lv.getValAtLevel(level) != max)
+        return level;
+    }
+    return endLevel;
+  }
+
+  /** Converts a UnitNRShape shape to the corresponding type supported by this class, such as a Calendar/BigDecimal.
+   * This is the reverse of {@link #toUnitShape(Object)}.
+   */
+  public abstract Object toObject(UnitNRShape shape);
+
+  /** A string representation of the UnitNRShape that is parse-able by {@link #parseUnitShape(String)}. */
+  protected abstract String toString(UnitNRShape lv);
+
+  protected static String toStringUnitRaw(UnitNRShape lv) {
+    StringBuilder buf = new StringBuilder(100);
+    buf.append('[');
+    for (int level = 1; level <= lv.getLevel(); level++) {
+      buf.append(lv.getValAtLevel(level)).append(',');
+    }
+    buf.setLength(buf.length()-1);//chop off ','
+    buf.append(']');
+    return buf.toString();
+  }
+
+  /** Detects a range pattern and parses it, otherwise it's parsed as one shape via
+   * {@link #parseUnitShape(String)}.  The range pattern looks like this BNF:
+   * <pre>
+   *   '[' + parseShapeLV + ' TO ' + parseShapeLV + ']'
+   * </pre>
+   * It's the same thing as the toString() of the range shape, notwithstanding range optimization.
+   *
+   * @param str not null or empty
+   * @return not null
+   * @throws java.text.ParseException If there is a problem
+   */
+  public NRShape parseShape(String str) throws ParseException {
+    if (str == null || str.isEmpty())
+      throw new IllegalArgumentException("str is null or blank");
+    if (str.charAt(0) == '[') {
+      if (str.charAt(str.length()-1) != ']')
+        throw new ParseException("If starts with [ must end with ]; got "+str, str.length()-1);
+      int middle = str.indexOf(" TO ");
+      if (middle < 0)
+        throw new ParseException("If starts with [ must contain ' TO '; got "+str, -1);
+      String leftStr = str.substring(1, middle);
+      String rightStr = str.substring(middle + " TO ".length(), str.length()-1);
+      return toRangeShape(parseUnitShape(leftStr), parseUnitShape(rightStr));
+    } else if (str.charAt(0) == '{') {
+      throw new ParseException("Exclusive ranges not supported; got "+str, 0);
+    } else {
+      return parseUnitShape(str);
+    }
+  }
+
+  /** Parse a String to a UnitNRShape. "*" should be the full-range (level 0 shape). */
+  protected abstract UnitNRShape parseUnitShape(String str) throws ParseException;
+
+
+  //
+  //    UnitNRShape
+  //
+
+  /**
+   * A unit value Shape implemented as a stack of numbers, one for each level in the prefix tree. It directly
+   * corresponds to a {@link Cell}.  Spatially speaking, it's analogous to a Point but 1D and has some precision width.
+   * @lucene.experimental
+   */
+  public static interface UnitNRShape extends NRShape, Comparable<UnitNRShape> {
+    //note: formerly known as LevelledValue; thus some variables still use 'lv'
+
+    /** Get the prefix tree level, the higher the more precise. 0 means the world (universe). */
+    int getLevel();
+    /** Gets the value at the specified level of this unit. level must be &gt;= 0 and &lt;= getLevel(). */
+    int getValAtLevel(int level);
+    /** Gets an ancestor at the specified level. It shares state, so you may want to clone() it. */
+    UnitNRShape getShapeAtLevel(int level);
+    @Override
+    UnitNRShape roundToLevel(int targetLevel);
+
+    /** Deep clone */
+    UnitNRShape clone();
+  }
+
+  /** Compares a to b, returning less than 0, 0, or greater than 0, if a is less than, equal to, or
+   * greater than b, respectively, up to their common prefix (i.e. only min(a.levels,b.levels) are compared).
+   * @lucene.internal */
+  protected static int comparePrefix(UnitNRShape a, UnitNRShape b) {
+    int minLevel = Math.min(a.getLevel(), b.getLevel());
+    for (int level = 1; level <= minLevel; level++) {
+      int diff = a.getValAtLevel(level) - b.getValAtLevel(level);
+      if (diff != 0)
+        return diff;
+    }
+    return 0;
+  }
+
+
+  //
+  //    SpanUnitsNRShape
+  //
+
+  /** A range Shape; based on a pair of {@link org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape}.
+   * Spatially speaking, it's analogous to a Rectangle but 1D. It might have been named with Range in the name but it
+   * may be confusing since even the {@link org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape}
+   * is in some sense a range.
+   * @lucene.experimental */
+  public class SpanUnitsNRShape implements NRShape {
+
+    private final UnitNRShape minLV, maxLV;
+    private final int lastLevelInCommon;//computed; not part of identity
+
+    /** Don't call directly; see
+     * {@link #toRangeShape(org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape, org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape)}. */
+    private SpanUnitsNRShape(UnitNRShape minLV, UnitNRShape maxLV) {
+      this.minLV = minLV;
+      this.maxLV = maxLV;
+
+      //calc lastLevelInCommon
+      int level = 1;
+      for (; level <= minLV.getLevel() && level <= maxLV.getLevel(); level++) {
+        if (minLV.getValAtLevel(level) != maxLV.getValAtLevel(level))
+          break;
+      }
+      lastLevelInCommon = level - 1;
+    }
+
+    @Override
+    public SpatialContext getContext() {
+      return DUMMY_CTX;
+    }
+
+    public UnitNRShape getMinUnit() { return minLV; }
+
+    public UnitNRShape getMaxUnit() { return maxLV; }
+
+    /** How many levels are in common between minUnit and maxUnit, not including level 0. */
+    private int getLevelsInCommon() { return lastLevelInCommon; }
+
+    @Override
+    public NRShape roundToLevel(int targetLevel) {
+      return toRangeShape(minLV.roundToLevel(targetLevel), maxLV.roundToLevel(targetLevel));
+    }
+
+    @Override
+    public SpatialRelation relate(Shape shape) {
+//      if (shape instanceof UnitNRShape)
+//        return relate((UnitNRShape)shape);
+      if (shape instanceof SpanUnitsNRShape)
+        return relate((SpanUnitsNRShape) shape);
+      return shape.relate(this).transpose();//probably a UnitNRShape
+    }
+
+    public SpatialRelation relate(SpanUnitsNRShape ext) {
+      //This logic somewhat mirrors RectangleImpl.relate_range()
+      int extMin_intMax = comparePrefix(ext.getMinUnit(), getMaxUnit());
+      if (extMin_intMax > 0)
+        return SpatialRelation.DISJOINT;
+      int extMax_intMin = comparePrefix(ext.getMaxUnit(), getMinUnit());
+      if (extMax_intMin < 0)
+        return SpatialRelation.DISJOINT;
+      int extMin_intMin = comparePrefix(ext.getMinUnit(), getMinUnit());
+      int extMax_intMax = comparePrefix(ext.getMaxUnit(), getMaxUnit());
+      if ((extMin_intMin > 0 || extMin_intMin == 0 && ext.getMinUnit().getLevel() >= getMinUnit().getLevel())
+          && (extMax_intMax < 0 || extMax_intMax == 0 && ext.getMaxUnit().getLevel() >= getMaxUnit().getLevel()))
+        return SpatialRelation.CONTAINS;
+      if ((extMin_intMin < 0 || extMin_intMin == 0 && ext.getMinUnit().getLevel() <= getMinUnit().getLevel())
+          && (extMax_intMax > 0 || extMax_intMax == 0 && ext.getMaxUnit().getLevel() <= getMaxUnit().getLevel()))
+        return SpatialRelation.WITHIN;
+      return SpatialRelation.INTERSECTS;
+    }
+
+    @Override
+    public Rectangle getBoundingBox() { throw new UnsupportedOperationException(); }
+
+    @Override
+    public boolean hasArea() { return true; }
+
+    @Override
+    public double getArea(SpatialContext spatialContext) { throw new UnsupportedOperationException(); }
+
+    @Override
+    public Point getCenter() { throw new UnsupportedOperationException(); }
+
+    @Override
+    public Shape getBuffered(double v, SpatialContext spatialContext) { throw new UnsupportedOperationException(); }
+
+    @Override
+    public boolean isEmpty() { return false; }
+
+    /** A deep clone. */
+    @Override
+    public SpanUnitsNRShape clone() {
+      return new SpanUnitsNRShape(minLV.clone(), maxLV.clone());
+    }
+
+    @Override
+    public String toString() {
+      return "[" + NumberRangePrefixTree.this.toString(minLV) + " TO "
+          + NumberRangePrefixTree.this.toString(maxLV) + "]";
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+
+      SpanUnitsNRShape spanShape = (SpanUnitsNRShape) o;
+
+      if (!maxLV.equals(spanShape.maxLV)) return false;
+      if (!minLV.equals(spanShape.minLV)) return false;
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int result = minLV.hashCode();
+      result = 31 * result + maxLV.hashCode();
+      return result;
+    }
+  }// class SpanUnitsNRShape
+
+  //
+  //    NumberRangePrefixTree
+  //
+
+  protected final int[] maxSubCellsByLevel;
+  protected final int[] termLenByLevel;
+  protected final int[] levelByTermLen;
+  protected final int maxTermLen; // how long could cell.getToken... (that is a leaf) possibly be?
+
+  protected NumberRangePrefixTree(int[] maxSubCellsByLevel) {
+    super(DUMMY_CTX, maxSubCellsByLevel.length);
+    this.maxSubCellsByLevel = maxSubCellsByLevel;
+
+    // Fill termLenByLevel
+    this.termLenByLevel = new int[maxLevels + 1];
+    termLenByLevel[0] = 0;
+    final int MAX_STATES = 1 << 15;//1 bit less than 2 bytes
+    for (int level = 1; level <= maxLevels; level++) {
+      final int states = maxSubCellsByLevel[level - 1];
+      if (states >= MAX_STATES || states <= 1) {
+        throw new IllegalArgumentException("Max states is "+MAX_STATES+", given "+states+" at level "+level);
+      }
+      boolean twoBytes = states >= 256;
+      termLenByLevel[level] = termLenByLevel[level-1] + (twoBytes ? 2 : 1);
+    }
+    maxTermLen = termLenByLevel[maxLevels] + 1;// + 1 for leaf byte
+
+    // Fill levelByTermLen
+    levelByTermLen = new int[maxTermLen];
+    levelByTermLen[0] = 0;
+    for (int level = 1; level < termLenByLevel.length; level++) {
+      int termLen = termLenByLevel[level];
+      int prevTermLen = termLenByLevel[level-1];
+      if (termLen - prevTermLen == 2) {//2 byte delta
+        //if the term doesn't completely cover this cell then it must be a leaf of the prior.
+        levelByTermLen[termLen-1] = -1;//won't be used; otherwise erroneous
+        levelByTermLen[termLen] = level;
+      } else {//1 byte delta
+        assert termLen - prevTermLen == 1;
+        levelByTermLen[termLen] = level;
+      }
+    }
+
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName();
+  }
+
+  @Override
+  public int getLevelForDistance(double dist) {
+    //note: it might be useful to compute which level has a raw width (counted in
+    // bottom units, e.g. milliseconds), that covers the provided dist in those units?
+    return maxLevels; // thus always use full precision. We don't do approximations in this tree/strategy.
+    //throw new UnsupportedOperationException("Not applicable.");
+  }
+
+  @Override
+  public double getDistanceForLevel(int level) {
+    //note: we could compute this... should we?
+    throw new UnsupportedOperationException("Not applicable.");
+  }
+
+  protected UnitNRShape toShape(int[] valStack, int len) {
+    final NRCell[] cellStack = newCellStack(len);
+    for (int i = 0; i < len; i++) {
+      cellStack[i+1].resetCellWithCellNum(valStack[i]);
+    }
+    return cellStack[len];
+  }
+
+  @Override
+  public Cell getWorldCell() {
+    return newCellStack(maxLevels)[0];
+  }
+
+  protected NRCell[] newCellStack(int levels) {
+    final NRCell[] cellsByLevel = new NRCell[levels + 1];
+    final BytesRef term = new BytesRef(maxTermLen);
+    for (int level = 0; level <= levels; level++) {
+      cellsByLevel[level] = new NRCell(cellsByLevel,term,level);
+    }
+    return cellsByLevel;
+  }
+
+  @Override
+  public Cell readCell(BytesRef term, Cell scratch) {
+    if (scratch == null)
+      scratch = getWorldCell();
+
+    //We decode level #, leaf boolean, and populate bytes by reference. We don't decode the stack.
+
+    //reverse lookup term length to the level and hence the cell
+    NRCell[] cellsByLevel = ((NRCell) scratch).cellsByLevel;
+    boolean isLeaf = term.bytes[term.offset + term.length - 1] == 0;
+    int lenNoLeaf = isLeaf ? term.length - 1 : term.length;
+
+    NRCell result = cellsByLevel[levelByTermLen[lenNoLeaf]];
+    if (cellsByLevel[0].termBuf == null)
+      cellsByLevel[0].termBuf = result.term.bytes;//a kluge; see cell.ensureOwnTermBytes()
+    result.term.bytes = term.bytes;
+    result.term.offset = term.offset;
+    result.term.length = lenNoLeaf;//technically this isn't used but may help debugging
+    result.reset();
+    if (isLeaf)
+      result.setLeaf();
+
+    result.cellNumber = -1;//lazy decode flag
+
+    return result;
+  }
+
+  /** Returns the number of sub-cells beneath the given UnitNRShape. */
+  public int getNumSubCells(UnitNRShape lv) {
+    return maxSubCellsByLevel[lv.getLevel()];
+  }
+
+  //
+  //    NRCell
+  //
+
+  /** Most of the PrefixTree implementation is in this one class, which is both
+   * the Cell, the CellIterator, and the Shape to reduce object allocation. It's implemented as a re-used array/stack
+   * of Cells at adjacent levels, that all have a reference back to the cell array to traverse. They also share a common
+   * BytesRef for the term.
+   * @lucene.internal */
+  protected class NRCell extends CellIterator implements Cell, UnitNRShape {
+
+    //Shared: (TODO put this in a new class)
+    final NRCell[] cellsByLevel;
+    final BytesRef term;//AKA the token
+    byte[] termBuf;// see ensureOwnTermBytes(), only for cell0
+
+    //Cell state...
+    final int cellLevel; // assert levelStack[cellLevel] == this
+    int cellNumber; //relative to parent cell. It's unused for level 0. Starts at 0.
+
+    SpatialRelation cellShapeRel;
+    boolean cellIsLeaf;
+
+    //CellIterator state is defined further below
+
+    NRCell(NRCell[] cellsByLevel, BytesRef term, int cellLevel) {
+      this.cellsByLevel = cellsByLevel;
+      this.term = term;
+      this.cellLevel = cellLevel;
+      this.cellNumber = cellLevel == 0 ? 0 : -1;
+      this.cellIsLeaf = false;
+      assert cellsByLevel[cellLevel] == null;
+    }
+
+    /** Ensure we own term.bytes so that it's safe to modify. We detect via a kluge in which cellsByLevel[0].termBuf
+     * is non-null, which is a pre-allocated for use to replace term.bytes. */
+    void ensureOwnTermBytes() {
+      NRCell cell0 = cellsByLevel[0];
+      if (cell0.termBuf == null)
+        return;//we already own the bytes
+      System.arraycopy(term.bytes, term.offset, cell0.termBuf, 0, term.length);
+      term.bytes = cell0.termBuf;
+      term.offset = 0;
+      cell0.termBuf = null;
+    }
+
+    private void reset() {
+      this.cellIsLeaf = false;
+      this.cellShapeRel = null;
+    }
+
+    private void resetCellWithCellNum(int cellNumber) {
+      reset();
+
+      //update bytes
+      //  note: see lazyInitCellNumsFromBytes() for the reverse
+      if (cellNumber >= 0) {//valid
+        ensureOwnTermBytes();
+        int termLen = termLenByLevel[getLevel()];
+        boolean twoBytes = (termLen - termLenByLevel[getLevel()-1]) > 1;
+        if (twoBytes) {
+          //right 7 bits, plus 1 (may overflow to 8th bit which is okay)
+          term.bytes[termLen-2] = (byte) (cellNumber >> 7);
+          term.bytes[termLen-1] = (byte) ((cellNumber & 0x7F) + 1);
+        } else {
+          term.bytes[termLen-1] = (byte) (cellNumber+1);
+        }
+        assert term.bytes[termLen-1] != 0;
+        term.length = termLen;
+      }
+      this.cellNumber = cellNumber;
+    }
+
+    private void ensureDecoded() {
+      if (cellNumber >= 0)
+        return;
+      //Decode cell numbers from bytes. This is the inverse of resetCellWithCellNum().
+      for (int level = 1; level <= getLevel(); level++) {
+        NRCell cell = cellsByLevel[level];
+        int termLen = termLenByLevel[level];
+        boolean twoBytes = (termLen - termLenByLevel[level-1]) > 1;
+        if (twoBytes) {
+          int byteH = (term.bytes[term.offset + termLen - 2] & 0xFF);
+          int byteL = (term.bytes[term.offset + termLen - 1] & 0xFF);
+          assert byteL - 1 < (1<<7);
+          cell.cellNumber = (byteH << 7) + (byteL-1);
+          assert cell.cellNumber < 1<<15;
+        } else {
+          cell.cellNumber = (term.bytes[term.offset + termLen - 1] & 0xFF) - 1;
+          assert cell.cellNumber < 255;
+        }
+        cell.assertDecoded();
+      }
+    }
+
+    private void assertDecoded() {
+      assert cellNumber >= 0 : "Illegal state; ensureDecoded() wasn't called";
+    }
+
+    @Override // for Cell & for UnitNRShape
+    public int getLevel() {
+      return cellLevel;
+    }
+
+    @Override
+    public SpatialRelation getShapeRel() {
+      return cellShapeRel;
+    }
+
+    @Override
+    public void setShapeRel(SpatialRelation rel) {
+      cellShapeRel = rel;
+    }
+
+    @Override
+    public boolean isLeaf() {
+      return cellIsLeaf;
+    }
+
+    @Override
+    public void setLeaf() {
+      cellIsLeaf = true;
+    }
+
+    @Override
+    public UnitNRShape getShape() {
+      ensureDecoded();
+      return this;
+    }
+
+    @Override
+    public BytesRef getTokenBytesNoLeaf(BytesRef result) {
+      if (result == null)
+        result = new BytesRef();
+      result.bytes = term.bytes;
+      result.offset = term.offset;
+      result.length = termLenByLevel[cellLevel];
+      assert result.length <= term.length;
+      return result;
+    }
+
+    @Override
+    public BytesRef getTokenBytesWithLeaf(BytesRef result) {
+      ensureOwnTermBytes();//normally shouldn't do anything
+      result = getTokenBytesNoLeaf(result);
+      if (isLeaf()) {
+        result.bytes[result.length++] = 0;
+      }
+      return result;
+    }
+
+    @Override
+    public boolean isPrefixOf(Cell c) {
+      NRCell otherCell = (NRCell) c;
+      assert term != otherCell.term;
+      //trick to re-use bytesref; provided that we re-instate it
+      int myLastLen = term.length;
+      term.length = termLenByLevel[getLevel()];
+      int otherLastLen = otherCell.term.length;
+      otherCell.term.length = termLenByLevel[otherCell.getLevel()];
+      boolean answer = StringHelper.startsWith(otherCell.term, term);
+      term.length = myLastLen;
+      otherCell.term.length = otherLastLen;
+      return answer;
+    }
+
+    @Override
+    public int compareToNoLeaf(Cell fromCell) {
+      final NRCell nrCell = (NRCell) fromCell;
+      assert term != nrCell.term;
+      //trick to re-use bytesref; provided that we re-instate it
+      int myLastLen = term.length;
+      int otherLastLen = nrCell.term.length;
+      term.length = termLenByLevel[getLevel()];
+      nrCell.term.length = termLenByLevel[nrCell.getLevel()];
+      int answer = term.compareTo(nrCell.term);
+      term.length = myLastLen;
+      nrCell.term.length = otherLastLen;
+      return answer;
+    }
+
+    @Override
+    public CellIterator getNextLevelCells(Shape shapeFilter) {
+      ensureDecoded();
+      NRCell subCell = cellsByLevel[cellLevel + 1];
+      subCell.initIter(shapeFilter);
+      return subCell;
+    }
+
+    //----------- CellIterator
+
+    Shape iterFilter;//UnitNRShape or NRShape
+    boolean iterFirstIsIntersects;
+    boolean iterLastIsIntersects;
+    int iterFirstCellNumber;
+    int iterLastCellNumber;
+
+    private void initIter(Shape filter) {
+      cellNumber = -1;
+      if (filter instanceof UnitNRShape && ((UnitNRShape) filter).getLevel() == 0)
+        filter = null;//world means everything -- no filter
+      iterFilter = filter;
+
+      NRCell parent = getShapeAtLevel(getLevel() - 1);
+
+      // Initialize iter* members.
+
+      //no filter means all subcells
+      if (filter == null) {
+        iterFirstCellNumber = 0;
+        iterFirstIsIntersects = false;
+        iterLastCellNumber = getNumSubCells(parent) - 1;
+        iterLastIsIntersects = false;
+        return;
+      }
+
+      final UnitNRShape minLV;
+      final UnitNRShape maxLV;
+      final int lastLevelInCommon;//between minLV & maxLV
+      if (filter instanceof SpanUnitsNRShape) {
+        SpanUnitsNRShape spanShape = (SpanUnitsNRShape) iterFilter;
+        minLV = spanShape.getMinUnit();
+        maxLV = spanShape.getMaxUnit();
+        lastLevelInCommon = spanShape.getLevelsInCommon();
+      } else {
+        minLV = (UnitNRShape) iterFilter;
+        maxLV = minLV;
+        lastLevelInCommon = minLV.getLevel();
+      }
+
+      //fast path optimization that is usually true, but never first level
+      if (iterFilter == parent.iterFilter &&
+          (getLevel() <= lastLevelInCommon || parent.iterFirstCellNumber != parent.iterLastCellNumber)) {
+        //TODO benchmark if this optimization pays off. We avoid two comparePrefixLV calls.
+        if (parent.iterFirstIsIntersects && parent.cellNumber == parent.iterFirstCellNumber
+            && minLV.getLevel() >= getLevel()) {
+          iterFirstCellNumber = minLV.getValAtLevel(getLevel());
+          iterFirstIsIntersects = (minLV.getLevel() > getLevel());
+        } else {
+          iterFirstCellNumber = 0;
+          iterFirstIsIntersects = false;
+        }
+        if (parent.iterLastIsIntersects && parent.cellNumber == parent.iterLastCellNumber
+            && maxLV.getLevel() >= getLevel()) {
+          iterLastCellNumber = maxLV.getValAtLevel(getLevel());
+          iterLastIsIntersects = (maxLV.getLevel() > getLevel());
+        } else {
+          iterLastCellNumber = getNumSubCells(parent) - 1;
+          iterLastIsIntersects = false;
+        }
+        if (iterFirstCellNumber == iterLastCellNumber) {
+          if (iterLastIsIntersects)
+            iterFirstIsIntersects = true;
+          else if (iterFirstIsIntersects)
+            iterLastIsIntersects = true;
+        }
+        return;
+      }
+
+      //not common to get here, except for level 1 which always happens
+
+      int startCmp = comparePrefix(minLV, parent);
+      if (startCmp > 0) {//start comes after this cell
+        iterFirstCellNumber = 0;
+        iterFirstIsIntersects = false;
+        iterLastCellNumber = -1;//so ends early (no cells)
+        iterLastIsIntersects = false;
+        return;
+      }
+      int endCmp = comparePrefix(maxLV, parent);//compare to end cell
+      if (endCmp < 0) {//end comes before this cell
+        iterFirstCellNumber = 0;
+        iterFirstIsIntersects = false;
+        iterLastCellNumber = -1;//so ends early (no cells)
+        iterLastIsIntersects = false;
+        return;
+      }
+      if (startCmp < 0 || minLV.getLevel() < getLevel()) {
+        //start comes before...
+        iterFirstCellNumber = 0;
+        iterFirstIsIntersects = false;
+      } else {
+        iterFirstCellNumber = minLV.getValAtLevel(getLevel());
+        iterFirstIsIntersects = (minLV.getLevel() > getLevel());
+      }
+      if (endCmp > 0 || maxLV.getLevel() < getLevel()) {
+        //end comes after...
+        iterLastCellNumber = getNumSubCells(parent) - 1;
+        iterLastIsIntersects = false;
+      } else {
+        iterLastCellNumber = maxLV.getValAtLevel(getLevel());
+        iterLastIsIntersects = (maxLV.getLevel() > getLevel());
+      }
+      if (iterFirstCellNumber == iterLastCellNumber) {
+        if (iterLastIsIntersects)
+          iterFirstIsIntersects = true;
+        else if (iterFirstIsIntersects)
+          iterLastIsIntersects = true;
+      }
+    }
+
+    @Override
+    public boolean hasNext() {
+      thisCell = null;
+      if (nextCell != null)//calling hasNext twice in a row
+        return true;
+
+      if (cellNumber >= iterLastCellNumber)
+        return false;
+
+      resetCellWithCellNum(cellNumber < iterFirstCellNumber ? iterFirstCellNumber : cellNumber + 1);
+
+      boolean hasChildren =
+          (cellNumber == iterFirstCellNumber && iterFirstIsIntersects)
+              || (cellNumber == iterLastCellNumber && iterLastIsIntersects);
+
+      if (!hasChildren) {
+        setLeaf();
+        setShapeRel(SpatialRelation.WITHIN);
+      } else if (iterFirstCellNumber == iterLastCellNumber) {
+        setShapeRel(SpatialRelation.CONTAINS);
+      } else {
+        setShapeRel(SpatialRelation.INTERSECTS);
+      }
+
+      nextCell = this;
+      return true;
+    }
+
+    //TODO override nextFrom to be more efficient
+
+    //----------- UnitNRShape
+
+    @Override
+    public int getValAtLevel(int level) {
+      final int result = cellsByLevel[level].cellNumber;
+      assert result >= 0;//initialized (decoded)
+      return result;
+    }
+
+    @Override
+    public NRCell getShapeAtLevel(int level) {
+      assert level <= cellLevel;
+      return cellsByLevel[level];
+    }
+
+    @Override
+    public UnitNRShape roundToLevel(int targetLevel) {
+      if (getLevel() <= targetLevel) {
+        return this;
+      } else {
+        return getShapeAtLevel(targetLevel);
+      }
+    }
+
+    @Override
+    public SpatialRelation relate(Shape shape) {
+      assertDecoded();
+      if (shape == iterFilter && cellShapeRel != null)
+        return cellShapeRel;
+      if (shape instanceof UnitNRShape)
+        return relate((UnitNRShape)shape);
+      if (shape instanceof SpanUnitsNRShape)
+        return relate((SpanUnitsNRShape)shape);
+      return shape.relate(this).transpose();
+    }
+
+    public SpatialRelation relate(UnitNRShape lv) {
+      assertDecoded();
+      int cmp = comparePrefix(this, lv);
+      if (cmp != 0)
+        return SpatialRelation.DISJOINT;
+      if (getLevel() > lv.getLevel())
+        return SpatialRelation.WITHIN;
+      return SpatialRelation.CONTAINS;//or equals
+      //no INTERSECTS; that won't happen.
+    }
+
+    public SpatialRelation relate(SpanUnitsNRShape spanShape) {
+      assertDecoded();
+      int startCmp = comparePrefix(spanShape.getMinUnit(), this);
+      if (startCmp > 0) {//start comes after this cell
+        return SpatialRelation.DISJOINT;
+      }
+      int endCmp = comparePrefix(spanShape.getMaxUnit(), this);
+      if (endCmp < 0) {//end comes before this cell
+        return SpatialRelation.DISJOINT;
+      }
+      int nrMinLevel = spanShape.getMinUnit().getLevel();
+      int nrMaxLevel = spanShape.getMaxUnit().getLevel();
+      if ((startCmp < 0 || startCmp == 0 && nrMinLevel <= getLevel())
+          && (endCmp > 0 || endCmp == 0 && nrMaxLevel <= getLevel()))
+        return SpatialRelation.WITHIN;//or equals
+      //At this point it's Contains or Within.
+      if (startCmp != 0 || endCmp != 0)
+        return SpatialRelation.INTERSECTS;
+      //if min or max Level is less, it might be on the equivalent edge.
+      for (;nrMinLevel < getLevel(); nrMinLevel++) {
+        if (getValAtLevel(nrMinLevel + 1) != 0)
+          return SpatialRelation.INTERSECTS;
+      }
+      for (;nrMaxLevel < getLevel(); nrMaxLevel++) {
+        if (getValAtLevel(nrMaxLevel + 1) != getNumSubCells(getShapeAtLevel(nrMaxLevel)) - 1)
+          return SpatialRelation.INTERSECTS;
+      }
+      return SpatialRelation.CONTAINS;
+    }
+
+    @Override
+    public UnitNRShape clone() {
+      //no leaf distinction; this is purely based on UnitNRShape
+      NRCell cell = (NRCell) readCell(getTokenBytesNoLeaf(null), null);
+      cell.ensureOwnTermBytes();
+      return cell.getShape();
+    }
+
+    @Override
+    public int compareTo(UnitNRShape o) {
+      assertDecoded();
+      //no leaf distinction; this is purely based on UnitNRShape
+      int cmp = comparePrefix(this, o);
+      if (cmp != 0) {
+        return cmp;
+      } else {
+        return getLevel() - o.getLevel();
+      }
+    }
+
+    @Override
+    public Rectangle getBoundingBox() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean hasArea() {
+      return true;
+    }
+
+    @Override
+    public double getArea(SpatialContext ctx) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Point getCenter() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Shape getBuffered(double distance, SpatialContext ctx) { throw new UnsupportedOperationException(); }
+
+    @Override
+    public boolean isEmpty() {
+      return false;
+    }
+
+    //------- Object
+
+    @Override
+    public boolean equals(Object obj) {
+      if (!(obj instanceof NRCell)) {
+        return false;
+      }
+      if (this == obj)
+        return true;
+      NRCell nrCell = (NRCell) obj;
+      assert term != nrCell.term;
+      if (getLevel() != nrCell.getLevel())
+        return false;
+      //trick to re-use bytesref; provided that we re-instate it
+      int myLastLen = term.length;
+      int otherLastLen = nrCell.term.length;
+      boolean answer = getTokenBytesNoLeaf(term).equals(nrCell.getTokenBytesNoLeaf(nrCell.term));
+      term.length = myLastLen;
+      nrCell.term.length = otherLastLen;
+      return answer;
+    }
+
+    @Override
+    public SpatialContext getContext() {
+      return DUMMY_CTX;
+    }
+
+    @Override
+    public int hashCode() {
+      //trick to re-use bytesref; provided that we re-instate it
+      int myLastLen = term.length;
+      int result = getTokenBytesNoLeaf(term).hashCode();
+      term.length = myLastLen;
+      return result;
+    }
+
+    @Override
+    public String toString() {
+      return NumberRangePrefixTree.this.toString(getShape());
+    }
+
+    /** Configure your IDE to use this. */
+    public String toStringDebug() {
+      String pretty = toString();
+      if (getLevel() == 0)
+        return pretty;
+      return toStringUnitRaw(this) + (isLeaf() ? "•" : "") + " " + pretty;
+    }
+
+  } // END OF NRCell
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5b028349/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java
new file mode 100644
index 0000000..6fe2bff
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java
@@ -0,0 +1,459 @@
+/*
+ * 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.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+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 com.spatial4j.core.shape.impl.RectangleImpl;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Uses a compact binary representation of 8 bytes to encode a spatial quad trie.
+ *
+ * The binary representation is as follows:
+ * <pre>
+ * CCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCCDDDDDL
+ *
+ * Where C = Cell bits (2 per quad)
+ *       D = Depth bits (5 with max of 29 levels)
+ *       L = isLeaf bit
+ * </pre>
+ *
+ * It includes a built-in "pruneLeafyBranches" setting (true by default) similar to
+ * {@link org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy#setPruneLeafyBranches(boolean)} although
+ * this one only prunes at the target detail level (where it has the most effect).  Usually you should disable RPT's
+ * prune, since it is very memory in-efficient.
+ *
+ * @lucene.experimental
+ */
+public class PackedQuadPrefixTree extends QuadPrefixTree {
+  public static final int MAX_LEVELS_POSSIBLE = 29;
+  protected static final byte[] QUAD = new byte[] {0x00, 0x01, 0x02, 0x03};
+
+  protected boolean leafyPrune = true;
+
+  /**
+   * Factory for creating {@link PackedQuadPrefixTree} instances with useful defaults.
+   */
+  public static class Factory extends QuadPrefixTree.Factory {
+    @Override
+    protected SpatialPrefixTree newSPT() {
+      return new PackedQuadPrefixTree(ctx, maxLevels != null ? maxLevels : MAX_LEVELS_POSSIBLE);
+    }
+  }
+
+  public PackedQuadPrefixTree(SpatialContext ctx, int maxLevels) {
+    super(ctx, maxLevels);
+    if (maxLevels > MAX_LEVELS_POSSIBLE) {
+      throw new IllegalArgumentException("maxLevels of " + maxLevels + " exceeds limit of " + MAX_LEVELS_POSSIBLE);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(maxLevels:" + maxLevels + ",ctx:" + ctx + ",prune:" + leafyPrune + ")";
+  }
+
+  @Override
+  public Cell getWorldCell() {
+    return new PackedQuadCell(0x0L);
+  }
+
+  @Override
+  public Cell getCell(Point p, int level) {
+    List<Cell> cells = new ArrayList<>(1);
+    build(xmid, ymid, 0, cells, 0x0L, ctx.makePoint(p.getX(), p.getY()), level);
+    return cells.get(0);//note cells could be longer if p on edge
+  }
+
+  protected void build(double x, double y, int level, List<Cell> matches, long term, Shape shape, int maxLevel) {
+    double w = levelW[level] / 2;
+    double h = levelH[level] / 2;
+
+    // Z-Order
+    // http://en.wikipedia.org/wiki/Z-order_%28curve%29
+    checkBattenberg(QUAD[0], x - w, y + h, level, matches, term, shape, maxLevel);
+    checkBattenberg(QUAD[1], x + w, y + h, level, matches, term, shape, maxLevel);
+    checkBattenberg(QUAD[2], x - w, y - h, level, matches, term, shape, maxLevel);
+    checkBattenberg(QUAD[3], x + w, y - h, level, matches, term, shape, maxLevel);
+  }
+
+  protected void checkBattenberg(byte quad, double cx, double cy, int level, List<Cell> matches,
+                               long term, Shape shape, int maxLevel) {
+    // short-circuit if we find a match for the point (no need to continue recursion)
+    if (shape instanceof Point && !matches.isEmpty())
+      return;
+    double w = levelW[level] / 2;
+    double h = levelH[level] / 2;
+
+    SpatialRelation v = shape.relate(ctx.makeRectangle(cx - w, cx + w, cy - h, cy + h));
+
+    if (SpatialRelation.DISJOINT == v) {
+      return;
+    }
+
+    // set bits for next level
+    term |= (((long)(quad))<<(64-(++level<<1)));
+    // increment level
+    term = ((term>>>1)+1)<<1;
+
+    if (SpatialRelation.CONTAINS == v || (level >= maxLevel)) {
+      matches.add(new PackedQuadCell(term, v.transpose()));
+    } else {// SpatialRelation.WITHIN, SpatialRelation.INTERSECTS
+      build(cx, cy, level, matches, term, shape, maxLevel);
+    }
+  }
+
+  @Override
+  public Cell readCell(BytesRef term, Cell scratch) {
+    PackedQuadCell cell = (PackedQuadCell) scratch;
+    if (cell == null)
+      cell = (PackedQuadCell) getWorldCell();
+    cell.readCell(term);
+    return cell;
+  }
+
+  @Override
+  public CellIterator getTreeCellIterator(Shape shape, int detailLevel) {
+    if (detailLevel > maxLevels) {
+      throw new IllegalArgumentException("detailLevel:" + detailLevel +" exceed max: " + maxLevels);
+    }
+    return new PrefixTreeIterator(shape, (short) detailLevel);
+  }
+
+  public boolean isPruneLeafyBranches() {
+    return leafyPrune;
+  }
+
+  /** Like {@link org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy#setPruneLeafyBranches(boolean)}
+   * but more memory efficient and only applies to the detailLevel, where it has the most effect. */
+  public void setPruneLeafyBranches( boolean pruneLeafyBranches ) {
+    this.leafyPrune = pruneLeafyBranches;
+  }
+
+  /** See binary representation in the javadocs of {@link PackedQuadPrefixTree}. */
+  protected class PackedQuadCell extends QuadCell {
+    private long term;
+
+    PackedQuadCell(long term) {
+      super(null, 0, 0);
+      this.term = term;
+      this.b_off = 0;
+      this.bytes = longToByteArray(this.term);
+      this.b_len = 8;
+      readLeafAdjust();
+    }
+
+    PackedQuadCell(long term, SpatialRelation shapeRel) {
+      this(term);
+      this.shapeRel = shapeRel;
+    }
+
+    @Override
+    protected void readCell(BytesRef bytes) {
+      shapeRel = null;
+      shape = null;
+      this.bytes = bytes.bytes;
+      this.b_off = bytes.offset;
+      this.b_len = (short) bytes.length;
+      this.term = longFromByteArray(this.bytes, bytes.offset);
+      readLeafAdjust();
+    }
+
+    private final int getShiftForLevel(final int level) {
+      return 64 - (level<<1);
+    }
+
+    public boolean isEnd(final int level, final int shift) {
+      return (term != 0x0L && ((((0x1L<<(level<<1))-1)-(term>>>shift)) == 0x0L));
+    }
+
+    /**
+     * Get the next cell in the tree without using recursion. descend parameter requests traversal to the child nodes,
+     * setting this to false will step to the next sibling.
+     * Note: This complies with lexicographical ordering, once you've moved to the next sibling there is no backtracking.
+     */
+    public PackedQuadCell nextCell(boolean descend) {
+      final int level = getLevel();
+      final int shift = getShiftForLevel(level);
+      // base case: can't go further
+      if ( (!descend && isEnd(level, shift)) || isEnd(maxLevels, getShiftForLevel(maxLevels))) {
+        return null;
+      }
+      long newTerm;
+      final boolean isLeaf = (term&0x1L)==0x1L;
+      // if descend requested && we're not at the maxLevel
+      if ((descend && !isLeaf && (level != maxLevels)) || level == 0) {
+        // simple case: increment level bits (next level)
+        newTerm = ((term>>>1)+0x1L)<<1;
+      } else {  // we're not descending or we can't descend
+        newTerm = term + (0x1L<<shift);
+        // we're at the last sibling...force descend
+        if (((term>>>shift)&0x3L) == 0x3L) {
+          // adjust level for number popping up
+          newTerm = ((newTerm>>>1) - (Long.numberOfTrailingZeros(newTerm>>>shift)>>>1))<<1;
+        }
+      }
+      return new PackedQuadCell(newTerm);
+    }
+
+    @Override
+    protected void readLeafAdjust() {
+      isLeaf = ((0x1L)&term) == 0x1L;
+      if (getLevel() == getMaxLevels()) {
+        isLeaf = true;
+      }
+    }
+
+    @Override
+    public BytesRef getTokenBytesWithLeaf(BytesRef result) {
+      if (isLeaf) {
+        term |= 0x1L;
+      }
+      return getTokenBytesNoLeaf(result);
+    }
+
+    @Override
+    public BytesRef getTokenBytesNoLeaf(BytesRef result) {
+      if (result == null)
+        return new BytesRef(bytes, b_off, b_len);
+      result.bytes = longToByteArray(this.term);
+      result.offset = 0;
+      result.length = result.bytes.length;
+      return result;
+    }
+
+    @Override
+    public int compareToNoLeaf(Cell fromCell) {
+      PackedQuadCell b = (PackedQuadCell) fromCell;
+      final long thisTerm = (((0x1L)&term) == 0x1L) ? term-1 : term;
+      final long fromTerm = (((0x1L)&b.term) == 0x1L) ? b.term-1 : b.term;
+      final int result = Long.compareUnsigned(thisTerm, fromTerm);
+      assert Math.signum(result)
+          == Math.signum(compare(longToByteArray(thisTerm), 0, 8, longToByteArray(fromTerm), 0, 8)); // TODO remove
+      return result;
+    }
+
+    @Override
+    public int getLevel() {
+      int l = (int)((term >>> 1)&0x1FL);
+      return l;
+    }
+
+    @Override
+    protected Collection<Cell> getSubCells() {
+      List<Cell> cells = new ArrayList<>(4);
+      PackedQuadCell pqc = (new PackedQuadCell(((term&0x1)==0x1) ? this.term-1 : this.term))
+          .nextCell(true);
+      cells.add(pqc);
+      cells.add((pqc = pqc.nextCell(false)));
+      cells.add((pqc = pqc.nextCell(false)));
+      cells.add(pqc.nextCell(false));
+      return cells;
+    }
+
+    @Override
+    protected QuadCell getSubCell(Point p) {
+      return (PackedQuadCell) PackedQuadPrefixTree.this.getCell(p, getLevel() + 1);//not performant!
+    }
+
+    @Override
+    public boolean isPrefixOf(Cell c) {
+      PackedQuadCell cell = (PackedQuadCell)c;
+      return (this.term == 0x0L) || isInternalPrefix(cell);
+    }
+
+    protected boolean isInternalPrefix(PackedQuadCell c) {
+      final int shift = 64 - (getLevel()<<1);
+      return ((term>>>shift)-(c.term>>>shift)) == 0x0L;
+    }
+
+    protected long concat(byte postfix) {
+      // extra leaf bit
+      return this.term | (((long)(postfix))<<((getMaxLevels()-getLevel()<<1)+6));
+    }
+
+    /**
+     * Constructs a bounding box shape out of the encoded cell
+     */
+    @Override
+    protected Rectangle makeShape() {
+      double xmin = PackedQuadPrefixTree.this.xmin;
+      double ymin = PackedQuadPrefixTree.this.ymin;
+      int level = getLevel();
+
+      byte b;
+      for (short l=0, i=1; l<level; ++l, ++i) {
+        b = (byte) ((term>>>(64-(i<<1))) & 0x3L);
+
+        switch (b) {
+          case 0x00:
+            ymin += levelH[l];
+            break;
+          case 0x01:
+            xmin += levelW[l];
+            ymin += levelH[l];
+            break;
+          case 0x02:
+            break;//nothing really
+          case 0x03:
+            xmin += levelW[l];
+            break;
+          default:
+            throw new RuntimeException("unexpected quadrant");
+        }
+      }
+
+      double width, height;
+      if (level > 0) {
+        width = levelW[level - 1];
+        height = levelH[level - 1];
+      } else {
+        width = gridW;
+        height = gridH;
+      }
+      return new RectangleImpl(xmin, xmin + width, ymin, ymin + height, ctx);
+    }
+
+    private long fromBytes(byte b1, byte b2, byte b3, byte b4, byte b5, byte b6, byte b7, byte b8) {
+      return ((long)b1 & 255L) << 56 | ((long)b2 & 255L) << 48 | ((long)b3 & 255L) << 40
+          | ((long)b4 & 255L) << 32 | ((long)b5 & 255L) << 24 | ((long)b6 & 255L) << 16
+          | ((long)b7 & 255L) << 8 | (long)b8 & 255L;
+    }
+
+    private byte[] longToByteArray(long value) {
+      byte[] result = new byte[8];
+      for(int i = 7; i >= 0; --i) {
+        result[i] = (byte)((int)(value & 255L));
+        value >>= 8;
+      }
+      return result;
+    }
+
+    private long longFromByteArray(byte[] bytes, int ofs) {
+      assert bytes.length >= 8;
+      return fromBytes(bytes[0+ofs], bytes[1+ofs], bytes[2+ofs], bytes[3+ofs],
+          bytes[4+ofs], bytes[5+ofs], bytes[6+ofs], bytes[7+ofs]);
+    }
+
+    /**
+     * Used for debugging, this will print the bits of the cell
+     */
+    @Override
+    public String toString() {
+      StringBuilder s = new StringBuilder(64);
+      final int numberOfLeadingZeros = Long.numberOfLeadingZeros(term);
+      for (int i = 0; i < numberOfLeadingZeros; i++) {
+        s.append('0');
+      }
+      if (term != 0)
+        s.append(Long.toBinaryString(term));
+      return s.toString();
+    }
+  } // PackedQuadCell
+
+  /** This is a streamlined version of TreeCellIterator, with built-in support to prune at detailLevel
+   * (but not recursively upwards). */
+  protected class PrefixTreeIterator extends CellIterator {
+    private Shape shape;
+    private PackedQuadCell thisCell;
+    private PackedQuadCell nextCell;
+
+    private short level;
+    private final short detailLevel;
+    private CellIterator pruneIter;
+
+    PrefixTreeIterator(Shape shape, short detailLevel) {
+      this.shape = shape;
+      this.thisCell = ((PackedQuadCell)(getWorldCell())).nextCell(true);
+      this.detailLevel = detailLevel;
+      this.nextCell = null;
+    }
+
+    @Override
+    public boolean hasNext() {
+      if (nextCell != null) {
+        return true;
+      }
+      SpatialRelation rel;
+      // loop until we're at the end of the quad tree or we hit a relation
+      while (thisCell != null) {
+        rel = thisCell.getShape().relate(shape);
+        if (rel == SpatialRelation.DISJOINT) {
+          thisCell = thisCell.nextCell(false);
+        } else { // within || intersects || contains
+          thisCell.setShapeRel(rel);
+          nextCell = thisCell;
+          if (rel == SpatialRelation.WITHIN) {
+            thisCell.setLeaf();
+            thisCell = thisCell.nextCell(false);
+          } else {  // intersects || contains
+            level = (short) (thisCell.getLevel());
+            if (level == detailLevel || pruned(rel)) {
+              thisCell.setLeaf();
+              if (shape instanceof Point) {
+                thisCell.setShapeRel(SpatialRelation.WITHIN);
+                thisCell = null;
+              } else {
+                thisCell = thisCell.nextCell(false);
+              }
+              break;
+            }
+            thisCell = thisCell.nextCell(true);
+          }
+          break;
+        }
+      }
+      return nextCell != null;
+    }
+
+    private boolean pruned(SpatialRelation rel) {
+      int leaves;
+      if (rel == SpatialRelation.INTERSECTS && leafyPrune && level == detailLevel - 1) {
+        for (leaves=0, pruneIter=thisCell.getNextLevelCells(shape); pruneIter.hasNext(); pruneIter.next(), ++leaves);
+        return leaves == 4;
+      }
+      return false;
+    }
+
+    @Override
+    public Cell next() {
+      if (nextCell == null) {
+        if (!hasNext()) {
+          throw new NoSuchElementException();
+        }
+      }
+      // overriding since this implementation sets thisCell in hasNext
+      Cell temp = nextCell;
+      nextCell = null;
+      return temp;
+    }
+
+    @Override
+    public void remove() {
+      //no-op
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5b028349/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java
new file mode 100644
index 0000000..48dac87
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java
@@ -0,0 +1,308 @@
+/*
+ * 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.io.PrintStream;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Locale;
+
+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.util.BytesRef;
+
+/**
+ * A {@link SpatialPrefixTree} which uses a
+ * <a href="http://en.wikipedia.org/wiki/Quadtree">quad tree</a> in which an
+ * indexed term will be generated for each cell, 'A', 'B', 'C', 'D'.
+ *
+ * @lucene.experimental
+ */
+public class QuadPrefixTree extends LegacyPrefixTree {
+
+  /**
+   * Factory for creating {@link QuadPrefixTree} instances with useful defaults
+   */
+  public static class Factory extends SpatialPrefixTreeFactory {
+
+    @Override
+    protected int getLevelForDistance(double degrees) {
+      QuadPrefixTree grid = new QuadPrefixTree(ctx, MAX_LEVELS_POSSIBLE);
+      return grid.getLevelForDistance(degrees);
+    }
+
+    @Override
+    protected SpatialPrefixTree newSPT() {
+      return new QuadPrefixTree(ctx,
+          maxLevels != null ? maxLevels : MAX_LEVELS_POSSIBLE);
+    }
+  }
+
+  public static final int MAX_LEVELS_POSSIBLE = 50;//not really sure how big this should be
+
+  public static final int DEFAULT_MAX_LEVELS = 12;
+  protected final double xmin;
+  protected final double xmax;
+  protected final double ymin;
+  protected final double ymax;
+  protected final double xmid;
+  protected final double ymid;
+
+  protected final double gridW;
+  public final double gridH;
+
+  final double[] levelW;
+  final double[] levelH;
+  final int[]    levelS; // side
+  final int[]    levelN; // number
+
+  public QuadPrefixTree(
+      SpatialContext ctx, Rectangle bounds, int maxLevels) {
+    super(ctx, maxLevels);
+    this.xmin = bounds.getMinX();
+    this.xmax = bounds.getMaxX();
+    this.ymin = bounds.getMinY();
+    this.ymax = bounds.getMaxY();
+
+    levelW = new double[maxLevels];
+    levelH = new double[maxLevels];
+    levelS = new int[maxLevels];
+    levelN = new int[maxLevels];
+
+    gridW = xmax - xmin;
+    gridH = ymax - ymin;
+    this.xmid = xmin + gridW/2.0;
+    this.ymid = ymin + gridH/2.0;
+    levelW[0] = gridW/2.0;
+    levelH[0] = gridH/2.0;
+    levelS[0] = 2;
+    levelN[0] = 4;
+
+    for (int i = 1; i < levelW.length; i++) {
+      levelW[i] = levelW[i - 1] / 2.0;
+      levelH[i] = levelH[i - 1] / 2.0;
+      levelS[i] = levelS[i - 1] * 2;
+      levelN[i] = levelN[i - 1] * 4;
+    }
+  }
+
+  public QuadPrefixTree(SpatialContext ctx) {
+    this(ctx, DEFAULT_MAX_LEVELS);
+  }
+
+  public QuadPrefixTree(
+      SpatialContext ctx, int maxLevels) {
+    this(ctx, ctx.getWorldBounds(), maxLevels);
+  }
+
+  @Override
+  public Cell getWorldCell() {
+    return new QuadCell(BytesRef.EMPTY_BYTES, 0, 0);
+  }
+
+  public void printInfo(PrintStream out) {
+    NumberFormat nf = NumberFormat.getNumberInstance(Locale.ROOT);
+    nf.setMaximumFractionDigits(5);
+    nf.setMinimumFractionDigits(5);
+    nf.setMinimumIntegerDigits(3);
+
+    for (int i = 0; i < maxLevels; i++) {
+      out.println(i + "]\t" + nf.format(levelW[i]) + "\t" + nf.format(levelH[i]) + "\t" +
+          levelS[i] + "\t" + (levelS[i] * levelS[i]));
+    }
+  }
+
+  @Override
+  public int getLevelForDistance(double dist) {
+    if (dist == 0)//short circuit
+      return maxLevels;
+    for (int i = 0; i < maxLevels-1; i++) {
+      //note: level[i] is actually a lookup for level i+1
+      if(dist > levelW[i] && dist > levelH[i]) {
+        return i+1;
+      }
+    }
+    return maxLevels;
+  }
+
+  @Override
+  public Cell getCell(Point p, int level) {
+    List<Cell> cells = new ArrayList<>(1);
+    build(xmid, ymid, 0, cells, new BytesRef(maxLevels+1), ctx.makePoint(p.getX(),p.getY()), level);
+    return cells.get(0);//note cells could be longer if p on edge
+  }
+
+  private void build(
+      double x,
+      double y,
+      int level,
+      List<Cell> matches,
+      BytesRef str,
+      Shape shape,
+      int maxLevel) {
+    assert str.length == level;
+    double w = levelW[level] / 2;
+    double h = levelH[level] / 2;
+
+    // Z-Order
+    // http://en.wikipedia.org/wiki/Z-order_%28curve%29
+    checkBattenberg('A', x - w, y + h, level, matches, str, shape, maxLevel);
+    checkBattenberg('B', x + w, y + h, level, matches, str, shape, maxLevel);
+    checkBattenberg('C', x - w, y - h, level, matches, str, shape, maxLevel);
+    checkBattenberg('D', x + w, y - h, level, matches, str, shape, maxLevel);
+
+    // possibly consider hilbert curve
+    // http://en.wikipedia.org/wiki/Hilbert_curve
+    // http://blog.notdot.net/2009/11/Damn-Cool-Algorithms-Spatial-indexing-with-Quadtrees-and-Hilbert-Curves
+    // if we actually use the range property in the query, this could be useful
+  }
+
+  protected void checkBattenberg(
+      char c,
+      double cx,
+      double cy,
+      int level,
+      List<Cell> matches,
+      BytesRef str,
+      Shape shape,
+      int maxLevel) {
+    assert str.length == level;
+    assert str.offset == 0;
+    double w = levelW[level] / 2;
+    double h = levelH[level] / 2;
+
+    int strlen = str.length;
+    Rectangle rectangle = ctx.makeRectangle(cx - w, cx + w, cy - h, cy + h);
+    SpatialRelation v = shape.relate(rectangle);
+    if (SpatialRelation.CONTAINS == v) {
+      str.bytes[str.length++] = (byte)c;//append
+      //str.append(SpatialPrefixGrid.COVER);
+      matches.add(new QuadCell(BytesRef.deepCopyOf(str), v.transpose()));
+    } else if (SpatialRelation.DISJOINT == v) {
+      // nothing
+    } else { // SpatialRelation.WITHIN, SpatialRelation.INTERSECTS
+      str.bytes[str.length++] = (byte)c;//append
+
+      int nextLevel = level+1;
+      if (nextLevel >= maxLevel) {
+        //str.append(SpatialPrefixGrid.INTERSECTS);
+        matches.add(new QuadCell(BytesRef.deepCopyOf(str), v.transpose()));
+      } else {
+        build(cx, cy, nextLevel, matches, str, shape, maxLevel);
+      }
+    }
+    str.length = strlen;
+  }
+
+  protected class QuadCell extends LegacyCell {
+
+    QuadCell(byte[] bytes, int off, int len) {
+      super(bytes, off, len);
+    }
+
+    QuadCell(BytesRef str, SpatialRelation shapeRel) {
+      this(str.bytes, str.offset, str.length);
+      this.shapeRel = shapeRel;
+    }
+
+    @Override
+    protected QuadPrefixTree getGrid() { return QuadPrefixTree.this; }
+
+    @Override
+    protected int getMaxLevels() { return maxLevels; }
+
+    @Override
+    protected Collection<Cell> getSubCells() {
+      BytesRef source = getTokenBytesNoLeaf(null);
+
+      List<Cell> cells = new ArrayList<>(4);
+      cells.add(new QuadCell(concat(source, (byte)'A'), null));
+      cells.add(new QuadCell(concat(source, (byte)'B'), null));
+      cells.add(new QuadCell(concat(source, (byte)'C'), null));
+      cells.add(new QuadCell(concat(source, (byte)'D'), null));
+      return cells;
+    }
+
+    protected BytesRef concat(BytesRef source, byte b) {
+      //+2 for new char + potential leaf
+      final byte[] buffer = Arrays.copyOfRange(source.bytes, source.offset, source.offset + source.length + 2);
+      BytesRef target = new BytesRef(buffer);
+      target.length = source.length;
+      target.bytes[target.length++] = b;
+      return target;
+    }
+
+    @Override
+    public int getSubCellsSize() {
+      return 4;
+    }
+
+    @Override
+    protected QuadCell getSubCell(Point p) {
+      return (QuadCell) QuadPrefixTree.this.getCell(p, getLevel() + 1);//not performant!
+    }
+
+    @Override
+    public Shape getShape() {
+      if (shape == null)
+        shape = makeShape();
+      return shape;
+    }
+
+    protected Rectangle makeShape() {
+      BytesRef token = getTokenBytesNoLeaf(null);
+      double xmin = QuadPrefixTree.this.xmin;
+      double ymin = QuadPrefixTree.this.ymin;
+
+      for (int i = 0; i < token.length; i++) {
+        byte c = token.bytes[token.offset + i];
+        switch (c) {
+          case 'A':
+            ymin += levelH[i];
+            break;
+          case 'B':
+            xmin += levelW[i];
+            ymin += levelH[i];
+            break;
+          case 'C':
+            break;//nothing really
+          case 'D':
+            xmin += levelW[i];
+            break;
+          default:
+            throw new RuntimeException("unexpected char: " + c);
+        }
+      }
+      int len = token.length;
+      double width, height;
+      if (len > 0) {
+        width = levelW[len-1];
+        height = levelH[len-1];
+      } else {
+        width = gridW;
+        height = gridH;
+      }
+      return ctx.makeRectangle(xmin, xmin + width, ymin, ymin + height);
+    }
+  }//QuadCell
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5b028349/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SingletonCellIterator.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SingletonCellIterator.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SingletonCellIterator.java
new file mode 100644
index 0000000..177b431
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SingletonCellIterator.java
@@ -0,0 +1,36 @@
+/*
+ * 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;
+
+/**
+ * A singleton (one Cell) instance of CellIterator.
+ *
+ * @lucene.internal
+ */
+class SingletonCellIterator extends CellIterator {
+
+  SingletonCellIterator(Cell cell) {
+    this.nextCell = cell;//preload nextCell
+  }
+
+  @Override
+  public boolean hasNext() {
+    thisCell = null;
+    return nextCell != null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5b028349/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java
new file mode 100644
index 0000000..8ead954
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java
@@ -0,0 +1,117 @@
+/*
+ * 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.context.SpatialContext;
+import com.spatial4j.core.shape.Shape;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A spatial Prefix Tree, or Trie, which decomposes shapes into prefixed strings
+ * at variable lengths corresponding to variable precision.  Each string
+ * corresponds to a rectangular spatial region.  This approach is
+ * also referred to "Grids", "Tiles", and "Spatial Tiers".
+ * <p>
+ * Implementations of this class should be thread-safe and immutable once
+ * initialized.
+ *
+ * @lucene.experimental
+ */
+public abstract class SpatialPrefixTree {
+
+  protected final int maxLevels;
+
+  protected final SpatialContext ctx;
+
+  public SpatialPrefixTree(SpatialContext ctx, int maxLevels) {
+    assert maxLevels > 0;
+    this.ctx = ctx;
+    this.maxLevels = maxLevels;
+  }
+
+  public SpatialContext getSpatialContext() {
+    return ctx;
+  }
+
+  public int getMaxLevels() {
+    return maxLevels;
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(maxLevels:" + maxLevels + ",ctx:" + ctx + ")";
+  }
+
+  /**
+   * Returns the level of the largest grid in which its longest side is less
+   * than or equal to the provided distance (in degrees). Consequently {@code
+   * dist} acts as an error epsilon declaring the amount of detail needed in the
+   * grid, such that you can get a grid with just the right amount of
+   * precision.
+   *
+   * @param dist {@code >= 0}
+   * @return level [1 to maxLevels]
+   */
+  public abstract int getLevelForDistance(double dist);
+
+  /**
+   * Given a cell having the specified level, returns the distance from opposite
+   * corners. Since this might vary depending on where the cell is, this method
+   * may over-estimate.
+   *
+   * @param level [1 to maxLevels]
+   * @return {@code > 0}
+   */
+  public abstract double getDistanceForLevel(int level);
+
+  /**
+   * Returns the level 0 cell which encompasses all spatial data. Equivalent to {@link #readCell(BytesRef,Cell)}
+   * with no bytes.
+   */
+  public abstract Cell getWorldCell(); //another possible name: getTopCell
+
+  /**
+   * This creates a new Cell (or re-using {@code scratch} if provided), initialized to the state as read
+   * by the bytes.
+   * Warning: An implementation may refer to the same byte array (no copy). If {@link Cell#setLeaf()} is
+   * subsequently called, it would then modify these bytes.
+   */
+  public abstract Cell readCell(BytesRef term, Cell scratch);
+
+  /**
+   * Gets the intersecting cells for the specified shape, without exceeding
+   * detail level. If a cell is within the query shape then it's marked as a
+   * leaf and none of its children are added. For cells at detailLevel, they are marked as
+   * leaves too, unless it's a point.
+   * <p>
+   * IMPORTANT: Cells returned from the iterator can be re-used for cells at the same level. So you can't simply
+   * iterate to subsequent cells and still refer to the former cell nor the bytes returned from the former cell, unless
+   * you know the former cell is a parent.
+   *
+   * @param shape       the shape; possibly null but the caller should liberally call
+   *  {@code remove()} if so.
+   * @param detailLevel the maximum detail level to get cells for
+   * @return the matching cells
+   */
+  public CellIterator getTreeCellIterator(Shape shape, int detailLevel) {
+    if (detailLevel > maxLevels) {
+      throw new IllegalArgumentException("detailLevel > maxLevels");
+    }
+    return new TreeCellIterator(shape, detailLevel, getWorldCell());
+  }
+
+}