You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ds...@apache.org on 2014/04/18 04:07:05 UTC

svn commit: r1588398 - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/spatial/ lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/ lucene/spatial/src/test/org/apache/lucene/spat...

Author: dsmiley
Date: Fri Apr 18 02:07:04 2014
New Revision: 1588398

URL: http://svn.apache.org/r1588398
Log:
LUCENE-5565: reverting spatial API change; leave in trunk for now

Removed:
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/CellTokenStream.java
Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/lucene/   (props changed)
    lucene/dev/branches/branch_4x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_4x/lucene/spatial/   (props changed)
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeTest.java

Modified: lucene/dev/branches/branch_4x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/CHANGES.txt?rev=1588398&r1=1588397&r2=1588398&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/lucene/CHANGES.txt Fri Apr 18 02:07:04 2014
@@ -166,8 +166,6 @@ API Changes
 
 * LUCENE-5543: Remove/deprecate Directory.fileExists (Mike McCandless)
 
-* LUCENE-5565: Refactor SpatialPrefixTree/Cell to not use Strings. (David Smiley)
-
 * LUCENE-5573: Move docvalues constants and helper methods to o.a.l.index.DocValues.
   (Dawid Weiss, Robert Muir)
 

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java?rev=1588398&r1=1588397&r2=1588398&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java Fri Apr 18 02:07:04 2014
@@ -25,6 +25,7 @@ import org.apache.lucene.spatial.prefix.
 import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.StringHelper;
 
 import java.io.IOException;
 import java.util.Iterator;
@@ -116,7 +117,7 @@ public abstract class AbstractVisitingPr
     protected final boolean hasIndexedLeaves;//if false then we can skip looking for them
 
     private VNode curVNode;//current pointer, derived from query shape
-    private BytesRef curVNodeTerm = new BytesRef();//curVNode.cell's term, without leaf
+    private BytesRef curVNodeTerm = new BytesRef();//curVNode.cell's term.
     private Cell scanCell;
 
     private BytesRef thisTerm;//the result of termsEnum.term()
@@ -170,7 +171,8 @@ public abstract class AbstractVisitingPr
         }
 
         //Seek to curVNode's cell (or skip if termsEnum has moved beyond)
-        curVNode.cell.getTokenBytesNoLeaf(curVNodeTerm);
+        curVNodeTerm.bytes = curVNode.cell.getTokenBytes();
+        curVNodeTerm.length = curVNodeTerm.bytes.length;
         int compare = termsEnum.getComparator().compare(thisTerm, curVNodeTerm);
         if (compare > 0) {
           // leap frog (termsEnum is beyond where we would otherwise seek)
@@ -213,7 +215,7 @@ public abstract class AbstractVisitingPr
       if (hasIndexedLeaves && cell.getLevel() != 0) {
         //If the next indexed term just adds a leaf marker ('+') to cell,
         // then add all of those docs
-        assert curVNode.cell.isWithin(curVNodeTerm, thisTerm);
+        assert StringHelper.startsWith(thisTerm, curVNodeTerm);//TODO refactor to use method on curVNode.cell
         scanCell = grid.getCell(thisTerm.bytes, thisTerm.offset, thisTerm.length, scanCell);
         if (scanCell.getLevel() == cell.getLevel() && scanCell.isLeaf()) {
           visitLeaf(scanCell);
@@ -263,7 +265,7 @@ public abstract class AbstractVisitingPr
      */
     protected void scan(int scanDetailLevel) throws IOException {
       for (;
-           thisTerm != null && curVNode.cell.isWithin(curVNodeTerm, thisTerm);
+           thisTerm != null && StringHelper.startsWith(thisTerm, curVNodeTerm);//TODO refactor to use method on curVNode.cell
            thisTerm = termsEnum.next()) {
         scanCell = grid.getCell(thisTerm.bytes, thisTerm.offset, thisTerm.length, scanCell);
 

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java?rev=1588398&r1=1588397&r2=1588398&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java Fri Apr 18 02:07:04 2014
@@ -83,7 +83,7 @@ public class ContainsPrefixTreeFilter ex
       super(context, acceptDocs);
     }
 
-    BytesRef termBytes = new BytesRef();//no leaf
+    BytesRef termBytes = new BytesRef();
     Cell nextCell;//see getLeafDocs
 
     /** This is the primary algorithm; recursive.  Returns null if finds none. */
@@ -130,15 +130,16 @@ public class ContainsPrefixTreeFilter ex
     }
 
     private boolean seekExact(Cell cell) throws IOException {
-      assert cell.getTokenBytesNoLeaf(null).compareTo(termBytes) > 0;
-      cell.getTokenBytesNoLeaf(termBytes);
+      assert new BytesRef(cell.getTokenBytes()).compareTo(termBytes) > 0;
+      termBytes.bytes = cell.getTokenBytes();
+      termBytes.length = termBytes.bytes.length;
       if (termsEnum == null)
         return false;
       return termsEnum.seekExact(termBytes);
     }
 
     private SmallDocSet getDocs(Cell cell, Bits acceptContains) throws IOException {
-      assert cell.getTokenBytesNoLeaf(null).equals(termBytes);
+      assert new BytesRef(cell.getTokenBytes()).equals(termBytes);
 
       return collectDocs(acceptContains);
     }
@@ -146,7 +147,7 @@ public class ContainsPrefixTreeFilter ex
     private Cell lastLeaf = null;//just for assertion
 
     private SmallDocSet getLeafDocs(Cell leafCell, Bits acceptContains) throws IOException {
-      assert leafCell.getTokenBytesNoLeaf(null).equals(termBytes);
+      assert new BytesRef(leafCell.getTokenBytes()).equals(termBytes);
       assert ! leafCell.equals(lastLeaf);//don't call for same leaf again
       lastLeaf = leafCell;
 

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java?rev=1588398&r1=1588397&r2=1588398&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java Fri Apr 18 02:07:04 2014
@@ -46,7 +46,7 @@ public class PointPrefixTreeFieldCachePr
   protected Point readShape(BytesRef term) {
     scanCell = grid.getCell(term.bytes, term.offset, term.length, scanCell);
     if (scanCell.getLevel() == grid.getMaxLevels() && !scanCell.isLeaf())
-      return scanCell.getShape().getCenter();
+      return scanCell.getCenter();
     return null;
   }
 }

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java?rev=1588398&r1=1588397&r2=1588398&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java Fri Apr 18 02:07:04 2014
@@ -19,6 +19,8 @@ package org.apache.lucene.spatial.prefix
 
 import com.spatial4j.core.shape.Point;
 import com.spatial4j.core.shape.Shape;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.index.FieldInfo;
@@ -30,6 +32,7 @@ import org.apache.lucene.spatial.prefix.
 import org.apache.lucene.spatial.query.SpatialArgs;
 import org.apache.lucene.spatial.util.ShapeFieldCacheDistanceValueSource;
 
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -123,12 +126,13 @@ public abstract class PrefixTreeStrategy
 
   public Field[] createIndexableFields(Shape shape, double distErr) {
     int detailLevel = grid.getLevelForDistance(distErr);
-    // note: maybe CellTokenStream should do this line, but it doesn't matter and it would create extra
-    // coupling
     List<Cell> cells = grid.getCells(shape, detailLevel, true, simplifyIndexedCells);//intermediates cells
 
+    //TODO is CellTokenStream supposed to be re-used somehow? see Uwe's comments:
+    //  http://code.google.com/p/lucene-spatial-playground/issues/detail?id=4
+
     Field field = new Field(getFieldName(),
-        new CellTokenStream().setCells(cells.iterator()), FIELD_TYPE);
+        new CellTokenStream(cells.iterator()), FIELD_TYPE);
     return new Field[]{field};
   }
 
@@ -143,6 +147,41 @@ public abstract class PrefixTreeStrategy
     FIELD_TYPE.freeze();
   }
 
+  /** Outputs the tokenString of a cell, and if its a leaf, outputs it again with the leaf byte. */
+  final static class CellTokenStream extends TokenStream {
+
+    private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+
+    private Iterator<Cell> iter = null;
+
+    public CellTokenStream(Iterator<Cell> tokens) {
+      this.iter = tokens;
+    }
+
+    CharSequence nextTokenStringNeedingLeaf = null;
+
+    @Override
+    public boolean incrementToken() {
+      clearAttributes();
+      if (nextTokenStringNeedingLeaf != null) {
+        termAtt.append(nextTokenStringNeedingLeaf);
+        termAtt.append((char) Cell.LEAF_BYTE);
+        nextTokenStringNeedingLeaf = null;
+        return true;
+      }
+      if (iter.hasNext()) {
+        Cell cell = iter.next();
+        CharSequence token = cell.getTokenString();
+        termAtt.append(token);
+        if (cell.isLeaf())
+          nextTokenStringNeedingLeaf = token;
+        return true;
+      }
+      return false;
+    }
+
+  }
+
   @Override
   public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
     PointPrefixTreeFieldCacheProvider p = provider.get( getFieldName() );

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java?rev=1588398&r1=1588397&r2=1588398&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java Fri Apr 18 02:07:04 2014
@@ -61,7 +61,7 @@ public class TermQueryPrefixTreeStrategy
     BytesRef[] terms = new BytesRef[cells.size()];
     int i = 0;
     for (Cell cell : cells) {
-      terms[i++] = cell.getTokenBytesNoLeaf(null);
+      terms[i++] = new BytesRef(cell.getTokenString());//TODO use cell.getTokenBytes()
     }
     return new TermsFilter(getFieldName(), terms);
   }

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java?rev=1588398&r1=1588397&r2=1588398&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java Fri Apr 18 02:07:04 2014
@@ -20,8 +20,6 @@ package org.apache.lucene.spatial.prefix
 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;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -29,45 +27,74 @@ import java.util.Collections;
 import java.util.List;
 
 /**
- * Represents a grid cell. These are not necessarily thread-safe, although calling {@link #getShape()} will
- * sufficiently prepare it to be so, if needed.
+ * Represents a grid cell. These are not necessarily thread-safe, although new
+ * Cell("") (world cell) must be.
  *
  * @lucene.experimental
  */
-public abstract class Cell {
+public abstract class Cell implements Comparable<Cell> {
+  public static final byte LEAF_BYTE = '+';//NOTE: must sort before letters & numbers
 
-  private static final byte LEAF_BYTE = '+';//NOTE: must sort before letters & numbers
-
-  //Arguably we could simply use a BytesRef, using an extra Object.
+  /*
+  Holds a byte[] and/or String representation of the cell. Both are lazy constructed from the other.
+  Neither contains the trailing leaf byte.
+   */
   private byte[] bytes;
   private int b_off;
   private int b_len;
 
+  private String token;//this is the only part of equality
+
   /**
    * When set via getSubCells(filter), it is the relationship between this cell
-   * and the given shape filter. Doesn't participate in shape equality.
+   * and the given shape filter.
    */
   protected SpatialRelation shapeRel;
 
-  /** Warning: Refers to the same bytes (no copy). If {@link #setLeaf()} is subsequently called then it
-   * may modify bytes. */
+  /**
+   * Always false for points. Otherwise, indicate no further sub-cells are going
+   * to be provided because shapeRel is WITHIN or maxLevels or a detailLevel is
+   * hit.
+   */
+  protected boolean leaf;
+
+  protected Cell(String token) {
+    this.token = token;
+    if (token.length() > 0 && token.charAt(token.length() - 1) == (char) LEAF_BYTE) {
+      this.token = token.substring(0, token.length() - 1);
+      setLeaf();
+    }
+
+    if (getLevel() == 0)
+      getShape();//ensure any lazy instantiation completes to make this threadsafe
+  }
+
   protected Cell(byte[] bytes, int off, int len) {
     this.bytes = bytes;
     this.b_off = off;
     this.b_len = len;
+    b_fixLeaf();
   }
 
-  /** Warning: Refers to the same bytes (no copy). If {@link #setLeaf()} is subsequently called then it
-   * may modify bytes. */
   public void reset(byte[] bytes, int off, int len) {
     assert getLevel() != 0;
+    token = null;
     shapeRel = null;
     this.bytes = bytes;
     this.b_off = off;
     this.b_len = len;
+    b_fixLeaf();
   }
 
-  protected abstract SpatialPrefixTree getGrid();
+  private void b_fixLeaf() {
+    //note that non-point shapes always have the maxLevels cell set with setLeaf
+    if (bytes[b_off + b_len - 1] == LEAF_BYTE) {
+      b_len--;
+      setLeaf();
+    } else {
+      leaf = false;
+    }
+  }
 
   public SpatialRelation getShapeRel() {
     return shapeRel;
@@ -78,68 +105,47 @@ public abstract class Cell {
    * further cells with this prefix for the shape (always true at maxLevels).
    */
   public boolean isLeaf() {
-    return (b_len > 0 && bytes[b_off + b_len - 1] == LEAF_BYTE);
+    return leaf;
   }
 
-  /** Modifies the bytes to reflect that this is a leaf. Warning: never invoke from a cell
-   * initialized to reference the same bytes from termsEnum, which should be treated as immutable.
-   * Note: not supported at level 0. */
+  /** Note: not supported at level 0. */
   public void setLeaf() {
     assert getLevel() != 0;
-    if (isLeaf())
-      return;
-    //if isn't big enough, we have to copy
-    if (bytes.length < b_off + b_len) {
-      //hopefully this copying doesn't happen too much (DWS: I checked and it doesn't seem to happen)
-      byte[] copy = new byte[b_len + 1];
-      System.arraycopy(bytes, b_off, copy, 0, b_len);
-      copy[b_len++] = LEAF_BYTE;
-      bytes = copy;
-      b_off = 0;
-    } else {
-      bytes[b_off + b_len++] = LEAF_BYTE;
-    }
+    leaf = true;
   }
 
   /**
-   * Returns the bytes for this cell.
-   * The result param is used to save object allocation, though it's bytes aren't used.
-   * @param result where the result goes, or null to create new
+   * Note: doesn't contain a trailing leaf byte.
    */
-  public BytesRef getTokenBytes(BytesRef result) {
-    if (result == null)
-      result = new BytesRef();
-    result.bytes = bytes;
-    result.offset = b_off;
-    result.length = b_len;
-    return result;
+  public String getTokenString() {
+    if (token == null) {
+      token = new String(bytes, b_off, b_len, SpatialPrefixTree.UTF8);
+    }
+    return token;
   }
 
   /**
-   * Returns the bytes for this cell, without leaf set. The bytes should sort before any
-   * cells that have the leaf set for the spatial location.
-   * The result param is used to save object allocation, though it's bytes aren't used.
-   * @param result where the result goes, or null to create new
+   * Note: doesn't contain a trailing leaf byte.
    */
-  public BytesRef getTokenBytesNoLeaf(BytesRef result) {
-    result = getTokenBytes(result);
-    if (isLeaf())
-      result.length--;
-    return result;
+  public byte[] getTokenBytes() {
+    if (bytes != null) {
+      if (b_off != 0 || b_len != bytes.length) {
+        throw new IllegalStateException("Not supported if byte[] needs to be recreated.");
+      }
+    } else {
+      bytes = token.getBytes(SpatialPrefixTree.UTF8);
+      b_off = 0;
+      b_len = bytes.length;
+    }
+    return bytes;
   }
 
-  /** Level 0 is the world (and has no parent), from then on a higher level means a smaller
-   * cell than the level before it.
-   */
   public int getLevel() {
-    return isLeaf() ? b_len - 1 : b_len;
+    return token != null ? token.length() : b_len;
   }
 
-  /** Gets the parent cell that contains this one. Don't call on the world cell. */
-  public Cell getParent() {
-    assert getLevel() > 0;
-    return getGrid().getCell(bytes, b_off, b_len - (isLeaf() ? 2 : 1));
-  }
+  //TODO add getParent() and update some algorithms to use this?
+  //public Cell getParent();
 
   /**
    * Like {@link #getSubCells()} but with the results filtered by a shape. If
@@ -190,6 +196,8 @@ public abstract class Cell {
    */
   public abstract Cell getSubCell(Point p);
 
+  //TODO Cell getSubCell(byte b)
+
   /**
    * Gets the cells at the next grid cell level that cover this cell.
    * Precondition: Never called when getLevel() == maxLevel.
@@ -203,45 +211,30 @@ public abstract class Cell {
    */
   public abstract int getSubCellsSize();
 
-  /** Gets the shape for this cell; typically a Rectangle. This method also serves to trigger any lazy
-   * loading needed to make the cell instance thread-safe.
-   */
   public abstract Shape getShape();
 
-  /** TODO remove once no longer used. */
   public Point getCenter() {
     return getShape().getCenter();
   }
 
   @Override
+  public int compareTo(Cell o) {
+    return getTokenString().compareTo(o.getTokenString());
+  }
+
+  @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 getTokenBytes(null).equals(cell.getTokenBytes(null));
-    } else {
-      return false;
-    }
+    return !(obj == null || !(obj instanceof Cell)) && getTokenString().equals(((Cell) obj).getTokenString());
   }
 
   @Override
   public int hashCode() {
-    return getTokenBytesNoLeaf(null).hashCode();
+    return getTokenString().hashCode();
   }
 
   @Override
   public String toString() {
-    //this method isn't "normally" called; just in asserts/tests
-    return getTokenBytes(null).utf8ToString();
+    return getTokenString() + (isLeaf() ? (char) LEAF_BYTE : "");
   }
 
-  /**
-   * Returns if the target term is within/underneath this cell; not necessarily a direct descendant.
-   * @param bytesNoLeaf must be getTokenBytesNoLeaf
-   * @param term the term
-   */
-  public boolean isWithin(BytesRef bytesNoLeaf, BytesRef term) {
-    assert bytesNoLeaf.equals(getTokenBytesNoLeaf(null));
-    return StringHelper.startsWith(term, bytesNoLeaf);
-  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java?rev=1588398&r1=1588397&r2=1588398&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java Fri Apr 18 02:07:04 2014
@@ -84,29 +84,18 @@ public class GeohashPrefixTree extends S
   }
 
   @Override
-  public Cell getCell(byte[] bytes, int offset, int len) {
-    return new GhCell(bytes, offset, len);
+  public Cell getCell(String token) {
+    return new GhCell(token);
   }
 
-  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;
+  @Override
+  public Cell getCell(byte[] bytes, int offset, int len) {
+    return new GhCell(bytes, offset, len);
   }
 
   class GhCell extends Cell {
-
-    private Shape shape;//cache
-    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())
-        this.geohash = geohash.substring(0, geohash.length() - 1);
+    GhCell(String token) {
+      super(token);
     }
 
     GhCell(byte[] bytes, int off, int len) {
@@ -114,12 +103,8 @@ public class GeohashPrefixTree extends S
     }
 
     @Override
-    protected SpatialPrefixTree getGrid() { return GeohashPrefixTree.this; }
-
-    @Override
     public void reset(byte[] bytes, int off, int len) {
       super.reset(bytes, off, len);
-      geohash = null;
       shape = null;
     }
 
@@ -140,26 +125,26 @@ public class GeohashPrefixTree extends S
 
     @Override
     public Cell getSubCell(Point p) {
-      return getGrid().getCell(p, getLevel() + 1);//not performant!
+      return GeohashPrefixTree.this.getCell(p, getLevel() + 1);//not performant!
     }
 
+    private Shape shape;//cache
+
     @Override
     public Shape getShape() {
       if (shape == null) {
-        shape = GeohashUtils.decodeBoundary(getGeohash(), getGrid().getSpatialContext());
+        shape = GeohashUtils.decodeBoundary(getGeohash(), ctx);
       }
       return shape;
     }
 
     @Override
     public Point getCenter() {
-      return GeohashUtils.decode(getGeohash(), getGrid().getSpatialContext());
+      return GeohashUtils.decode(getGeohash(), ctx);
     }
 
     private String getGeohash() {
-      if (geohash == null)
-        geohash = getTokenBytesNoLeaf(null).utf8ToString();
-      return geohash;
+      return getTokenString();
     }
 
   }//class GhCell

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java?rev=1588398&r1=1588397&r2=1588398&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java Fri Apr 18 02:07:04 2014
@@ -22,7 +22,6 @@ 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;
 
 import java.io.PrintStream;
 import java.text.NumberFormat;
@@ -143,11 +142,16 @@ public class QuadPrefixTree extends Spat
   @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);
+    build(xmid, ymid, 0, cells, new StringBuilder(), ctx.makePoint(p.getX(),p.getY()), level);
     return cells.get(0);//note cells could be longer if p on edge
   }
 
   @Override
+  public Cell getCell(String token) {
+    return new QuadCell(token);
+  }
+
+  @Override
   public Cell getCell(byte[] bytes, int offset, int len) {
     return new QuadCell(bytes, offset, len);
   }
@@ -157,10 +161,10 @@ public class QuadPrefixTree extends Spat
       double y,
       int level,
       List<Cell> matches,
-      BytesRef str,
+      StringBuilder str,
       Shape shape,
       int maxLevel) {
-    assert str.length == level;
+    assert str.length() == level;
     double w = levelW[level] / 2;
     double h = levelH[level] / 2;
 
@@ -183,50 +187,50 @@ public class QuadPrefixTree extends Spat
       double cy,
       int level,
       List<Cell> matches,
-      BytesRef str,
+      StringBuilder str,
       Shape shape,
       int maxLevel) {
-    assert str.length == level;
-    assert str.offset == 0;
+    assert str.length() == level;
     double w = levelW[level] / 2;
     double h = levelH[level] / 2;
 
-    int strlen = str.length;
+    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(c);
       //str.append(SpatialPrefixGrid.COVER);
-      matches.add(new QuadCell(BytesRef.deepCopyOf(str), v.transpose()));
+      matches.add(new QuadCell(str.toString(),v.transpose()));
     } else if (SpatialRelation.DISJOINT == v) {
       // nothing
     } else { // SpatialRelation.WITHIN, SpatialRelation.INTERSECTS
-      str.bytes[str.length++] = (byte)c;//append
+      str.append(c);
 
       int nextLevel = level+1;
       if (nextLevel >= maxLevel) {
         //str.append(SpatialPrefixGrid.INTERSECTS);
-        matches.add(new QuadCell(BytesRef.deepCopyOf(str), v.transpose()));
+        matches.add(new QuadCell(str.toString(),v.transpose()));
       } else {
         build(cx, cy, nextLevel, matches, str, shape, maxLevel);
       }
     }
-    str.length = strlen;
+    str.setLength(strlen);
   }
 
-  class QuadCell extends Cell{
+  class QuadCell extends Cell {
 
-    QuadCell(byte[] bytes, int off, int len) {
-      super(bytes, off, len);
+    public QuadCell(String token) {
+      super(token);
     }
 
-    QuadCell(BytesRef str, SpatialRelation shapeRel) {
-      this(str.bytes, str.offset, str.length);
+    public QuadCell(String token, SpatialRelation shapeRel) {
+      super(token);
       this.shapeRel = shapeRel;
     }
 
-    @Override
-    protected SpatialPrefixTree getGrid() { return QuadPrefixTree.this; }
+    QuadCell(byte[] bytes, int off, int len) {
+      super(bytes, off, len);
+    }
 
     @Override
     public void reset(byte[] bytes, int off, int len) {
@@ -236,26 +240,14 @@ public class QuadPrefixTree extends Spat
 
     @Override
     public Collection<Cell> getSubCells() {
-      BytesRef source = getTokenBytesNoLeaf(null);
-      BytesRef target = new BytesRef();
-
       List<Cell> cells = new ArrayList<>(4);
-      cells.add(new QuadCell(concat(source, (byte)'A', target), null));
-      cells.add(new QuadCell(concat(source, (byte)'B', target), null));
-      cells.add(new QuadCell(concat(source, (byte)'C', target), null));
-      cells.add(new QuadCell(concat(source, (byte)'D', target), null));
+      cells.add(new QuadCell(getTokenString()+"A"));
+      cells.add(new QuadCell(getTokenString()+"B"));
+      cells.add(new QuadCell(getTokenString()+"C"));
+      cells.add(new QuadCell(getTokenString()+"D"));
       return cells;
     }
 
-    private BytesRef concat(BytesRef source, byte b, BytesRef target) {
-      assert target.offset == 0;
-      target.bytes = new byte[source.length + 2];//+2 for new char + potential leaf
-      target.length = 0;
-      target.copyBytes(source);
-      target.bytes[target.length++] = b;
-      return target;
-    }
-
     @Override
     public int getSubCellsSize() {
       return 4;
@@ -276,30 +268,27 @@ public class QuadPrefixTree extends Spat
     }
 
     private Rectangle makeShape() {
-      BytesRef token = getTokenBytesNoLeaf(null);
+      String token = getTokenString();
       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);
+      for (int i = 0; i < token.length(); i++) {
+        char c = token.charAt(i);
+        if ('A' == c || 'a' == c) {
+          ymin += levelH[i];
+        } else if ('B' == c || 'b' == c) {
+          xmin += levelW[i];
+          ymin += levelH[i];
+        } else if ('C' == c || 'c' == c) {
+          // nothing really
+        }
+        else if('D' == c || 'd' == c) {
+          xmin += levelW[i];
+        } else {
+          throw new RuntimeException("unexpected char: " + c);
         }
       }
-      int len = token.length;
+      int len = token.length();
       double width, height;
       if (len > 0) {
         width = levelW[len-1];

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java?rev=1588398&r1=1588397&r2=1588398&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java Fri Apr 18 02:07:04 2014
@@ -21,12 +21,10 @@ import com.spatial4j.core.context.Spatia
 import com.spatial4j.core.shape.Point;
 import com.spatial4j.core.shape.Rectangle;
 import com.spatial4j.core.shape.Shape;
-import org.apache.lucene.util.BytesRef;
 
 import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
@@ -105,14 +103,14 @@ public abstract class SpatialPrefixTree 
   private transient Cell worldCell;//cached
 
   /**
-   * Returns the level 0 cell which encompasses all spatial data. Equivalent to {@link #getCell(byte[], int, int)} with
-   * no bytes. This cell is thread-safe, just like a spatial prefix grid is, although cells aren't
-   * generally thread-safe.
+   * Returns the level 0 cell which encompasses all spatial data. Equivalent to {@link #getCell(String)} with "".
+   * This cell is threadsafe, just like a spatial prefix grid is, although cells aren't
+   * generally threadsafe.
+   * TODO rename to getTopCell or is this fine?
    */
-  public Cell getWorldCell() {//another possible name: getTopCell
+  public Cell getWorldCell() {
     if (worldCell == null) {
-      worldCell = getCell(BytesRef.EMPTY_BYTES, 0, 0);
-      worldCell.getShape();//lazy load; make thread-safe
+      worldCell = getCell("");
     }
     return worldCell;
   }
@@ -121,6 +119,8 @@ public abstract class SpatialPrefixTree 
    * The cell for the specified token. The empty string should be equal to {@link #getWorldCell()}.
    * Precondition: Never called when token length > maxLevel.
    */
+  public abstract Cell getCell(String token);
+
   public abstract Cell getCell(byte[] bytes, int offset, int len);
 
   public final Cell getCell(byte[] bytes, int offset, int len, Cell target) {
@@ -215,23 +215,40 @@ public abstract class SpatialPrefixTree 
    * A Point-optimized implementation of
    * {@link #getCells(com.spatial4j.core.shape.Shape, int, boolean, boolean)}. That
    * method in facts calls this for points.
+   * <p/>
+   * This implementation depends on {@link #getCell(String)} being fast, as its
+   * called repeatedly when incPlarents is true.
    */
   public List<Cell> getCells(Point p, int detailLevel, boolean inclParents) {
     Cell cell = getCell(p, detailLevel);
-    assert !cell.isLeaf();
-    if (!inclParents || detailLevel == 1) {
+    if (!inclParents) {
       return Collections.singletonList(cell);
     }
 
-    //fill in reverse order to be sorted
-    Cell[] cells = new Cell[detailLevel];
-    for (int i = detailLevel-1; true; i--) {
-      cells[i] = cell;
-      if (i == 0)
-        break;
-      cell = cell.getParent();
+    String endToken = cell.getTokenString();
+    assert endToken.length() == detailLevel;
+    List<Cell> cells = new ArrayList<>(detailLevel);
+    for (int i = 1; i < detailLevel; i++) {
+      cells.add(getCell(endToken.substring(0, i)));//TODO refactor: add a cell.getParent()
     }
-    return Arrays.asList(cells);
+    cells.add(cell);
+    return cells;
   }
 
+  /**
+   * Will add the trailing leaf byte for leaves. This isn't particularly efficient.
+   * @deprecated TODO remove; not used and not interesting, don't need collection in & out
+   */
+  public static List<String> cellsToTokenStrings(Collection<Cell> cells) {
+    List<String> tokens = new ArrayList<>((cells.size()));
+    for (Cell cell : cells) {
+      final String token = cell.getTokenString();
+      if (cell.isLeaf()) {
+        tokens.add(token + (char) Cell.LEAF_BYTE);
+      } else {
+        tokens.add(token);
+      }
+    }
+    return tokens;
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeTest.java?rev=1588398&r1=1588397&r2=1588398&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeTest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeTest.java Fri Apr 18 02:07:04 2014
@@ -35,9 +35,6 @@ import org.apache.lucene.spatial.query.S
 import org.junit.Before;
 import org.junit.Test;
 
-import java.util.ArrayList;
-import java.util.List;
-
 public class SpatialPrefixTreeTest extends SpatialTestCase {
 
   //TODO plug in others and test them
@@ -59,10 +56,9 @@ public class SpatialPrefixTreeTest exten
     Cell c = trie.getWorldCell();
     assertEquals(0, c.getLevel());
     assertEquals(ctx.getWorldBounds(), c.getShape());
-    while (c.getLevel() < trie.getMaxLevels()) {
+    while(c.getLevel() < trie.getMaxLevels()) {
       prevC = c;
-      List<Cell> subCells = new ArrayList<>(c.getSubCells());
-      c = subCells.get(random().nextInt(subCells.size()-1));
+      c = c.getSubCells().iterator().next();//TODO random which one?
       
       assertEquals(prevC.getLevel()+1,c.getLevel());
       Rectangle prevNShape = (Rectangle) prevC.getShape();