You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rj...@apache.org on 2015/03/31 07:22:50 UTC

svn commit: r1670257 [14/39] - in /lucene/dev/branches/lucene6271: ./ dev-tools/ dev-tools/idea/.idea/libraries/ dev-tools/scripts/ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneou...

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java Tue Mar 31 05:22:40 2015
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.Iterator;
 
 import com.spatial4j.core.shape.Shape;
+import com.spatial4j.core.shape.SpatialRelation;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.DocIdSet;
@@ -53,13 +54,11 @@ public abstract class AbstractVisitingPr
   //  least it would just make things more complicated.
 
   protected final int prefixGridScanLevel;//at least one less than grid.getMaxLevels()
-  protected final boolean hasIndexedLeaves;
 
   public AbstractVisitingPrefixTreeFilter(Shape queryShape, String fieldName, SpatialPrefixTree grid,
-                                          int detailLevel, int prefixGridScanLevel, boolean hasIndexedLeaves) {
+                                          int detailLevel, int prefixGridScanLevel) {
     super(queryShape, fieldName, grid, detailLevel);
     this.prefixGridScanLevel = Math.max(0, Math.min(prefixGridScanLevel, grid.getMaxLevels() - 1));
-    this.hasIndexedLeaves = hasIndexedLeaves;
     assert detailLevel <= grid.getMaxLevels();
   }
 
@@ -90,7 +89,7 @@ public abstract class AbstractVisitingPr
    * which by default finds the top cells that intersect {@code queryShape}. If
    * there isn't an indexed cell for a corresponding cell returned for this
    * method then it's short-circuited until it finds one, at which point
-   * {@link #visit(org.apache.lucene.spatial.prefix.tree.Cell)} is called. At
+   * {@link #visitPrefix(org.apache.lucene.spatial.prefix.tree.Cell)} is called. At
    * some depths, of the tree, the algorithm switches to a scanning mode that
    * calls {@link #visitScanned(org.apache.lucene.spatial.prefix.tree.Cell)}
    * for each leaf cell found.
@@ -114,10 +113,6 @@ public abstract class AbstractVisitingPr
     does act as a short-circuit.  So maybe do some percent of the time or when the level
     is above some threshold.
 
-  * Once we don't have redundant non-leaves indexed with leaf cells (LUCENE-4942), we can
-    sometimes know to call next() instead of seek() if we're processing a leaf cell that
-    didn't have a corresponding non-leaf.
-
   */
 
     //
@@ -125,10 +120,10 @@ public abstract class AbstractVisitingPr
     //
 
     private VNode curVNode;//current pointer, derived from query shape
-    private BytesRef curVNodeTerm = new BytesRef();//curVNode.cell's term, without leaf
-    private Cell scanCell;
+    private BytesRef curVNodeTerm = new BytesRef();//curVNode.cell's term, without leaf. in main loop only
 
     private BytesRef thisTerm;//the result of termsEnum.term()
+    private Cell indexedCell;//Cell wrapper of thisTerm. Always updated when thisTerm is.
 
     public VisitorTemplate(LeafReaderContext context, Bits acceptDocs) throws IOException {
       super(context, acceptDocs);
@@ -138,9 +133,9 @@ public abstract class AbstractVisitingPr
       assert curVNode == null : "Called more than once?";
       if (termsEnum == null)
         return null;
-      //advance
-      if ((thisTerm = termsEnum.next()) == null)
-        return null; // all done
+      if (!nextTerm()) {//advances
+        return null;
+      }
 
       curVNode = new VNode(null);
       curVNode.reset(grid.getWorldCell());
@@ -177,30 +172,46 @@ public abstract class AbstractVisitingPr
         }
 
         //Seek to curVNode's cell (or skip if termsEnum has moved beyond)
-        curVNode.cell.getTokenBytesNoLeaf(curVNodeTerm);
-        int compare = thisTerm.compareTo(curVNodeTerm);
+        final int compare = indexedCell.compareToNoLeaf(curVNode.cell);
         if (compare > 0) {
-          // leap frog (termsEnum is beyond where we would otherwise seek)
-          assert ! context.reader().terms(fieldName).iterator(null).seekExact(curVNodeTerm) : "should be absent";
-        } else {
-          if (compare < 0) {
-            // Seek !
-            TermsEnum.SeekStatus seekStatus = termsEnum.seekCeil(curVNodeTerm);
-            if (seekStatus == TermsEnum.SeekStatus.END)
-              break; // all done
-            thisTerm = termsEnum.term();
-            if (seekStatus == TermsEnum.SeekStatus.NOT_FOUND) {
-              continue; // leap frog
-            }
-          }
-          // Visit!
-          boolean descend = visit(curVNode.cell);
-          //advance
-          if ((thisTerm = termsEnum.next()) == null)
+          // The indexed cell is after; continue loop to next query cell
+          continue;
+        }
+        if (compare < 0) {
+          // The indexed cell is before; seek ahead to query cell:
+          //      Seek !
+          curVNode.cell.getTokenBytesNoLeaf(curVNodeTerm);
+          TermsEnum.SeekStatus seekStatus = termsEnum.seekCeil(curVNodeTerm);
+          if (seekStatus == TermsEnum.SeekStatus.END)
             break; // all done
-          if (descend)
-            addIntersectingChildren();
+          thisTerm = termsEnum.term();
+          indexedCell = grid.readCell(thisTerm, indexedCell);
+          if (seekStatus == TermsEnum.SeekStatus.NOT_FOUND) {
+            // Did we find a leaf of the cell we were looking for or something after?
+            if (!indexedCell.isLeaf() || indexedCell.compareToNoLeaf(curVNode.cell) != 0)
+              continue; // The indexed cell is after; continue loop to next query cell
+          }
+        }
+        // indexedCell == queryCell (disregarding leaf).
 
+        // If indexedCell is a leaf then there's no prefix (prefix sorts before) -- just visit and continue
+        if (indexedCell.isLeaf()) {
+          visitLeaf(indexedCell);//TODO or query cell? Though shouldn't matter.
+          if (!nextTerm()) break;
+          continue;
+        }
+        // If a prefix (non-leaf) then visit; see if we descend.
+        final boolean descend = visitPrefix(curVNode.cell);//need to use curVNode.cell not indexedCell
+        if (!nextTerm()) break;
+        // Check for adjacent leaf with the same prefix
+        if (indexedCell.isLeaf() && indexedCell.getLevel() == curVNode.cell.getLevel()) {
+          visitLeaf(indexedCell);//TODO or query cell? Though shouldn't matter.
+          if (!nextTerm()) break;
+        }
+
+
+        if (descend) {
+          addIntersectingChildren();
         }
 
       }//main loop
@@ -208,7 +219,7 @@ public abstract class AbstractVisitingPr
       return finish();
     }
 
-    /** Called initially, and whenever {@link #visit(org.apache.lucene.spatial.prefix.tree.Cell)}
+    /** Called initially, and whenever {@link #visitPrefix(org.apache.lucene.spatial.prefix.tree.Cell)}
      * returns true. */
     private void addIntersectingChildren() throws IOException {
       assert thisTerm != null;
@@ -216,20 +227,6 @@ public abstract class AbstractVisitingPr
       if (cell.getLevel() >= detailLevel)
         throw new IllegalStateException("Spatial logic error");
 
-      //Check for adjacent leaf (happens for indexed non-point shapes)
-      if (hasIndexedLeaves && cell.getLevel() != 0) {
-        //If the next indexed term just adds a leaf marker to cell,
-        // then add all of those docs
-        scanCell = grid.readCell(thisTerm, scanCell);
-        assert curVNode.cell.isPrefixOf(scanCell) : "missing leaf or descendants";
-        if (scanCell.getLevel() == cell.getLevel() && scanCell.isLeaf()) {
-          visitLeaf(scanCell);
-          //advance
-          if ((thisTerm = termsEnum.next()) == null)
-            return; // all done
-        }
-      }
-
       //Decide whether to continue to divide & conquer, or whether it's time to
       // scan through terms beneath this cell.
       // Scanning is a performance optimization trade-off.
@@ -269,22 +266,22 @@ public abstract class AbstractVisitingPr
      * #visitScanned(org.apache.lucene.spatial.prefix.tree.Cell)}.
      */
     protected void scan(int scanDetailLevel) throws IOException {
-      for ( ;
-          thisTerm != null;
-          thisTerm = termsEnum.next()) {
-        scanCell = grid.readCell(thisTerm, scanCell);
-        if (!curVNode.cell.isPrefixOf(scanCell))
-          break;
-
-        int termLevel = scanCell.getLevel();
-        if (termLevel < scanDetailLevel) {
-          if (scanCell.isLeaf())
-            visitScanned(scanCell);
-        } else if (termLevel == scanDetailLevel) {
-          if (!scanCell.isLeaf())//LUCENE-5529
-            visitScanned(scanCell);
+      //note: this can be a do-while instead in 6x; 5x has a back-compat with redundant leaves -- LUCENE-4942
+      while (curVNode.cell.isPrefixOf(indexedCell)) {
+        if (indexedCell.getLevel() == scanDetailLevel
+            || (indexedCell.getLevel() < scanDetailLevel && indexedCell.isLeaf())) {
+          visitScanned(indexedCell);
         }
-      }//term loop
+        //advance
+        if (!nextTerm()) break;
+      }
+    }
+
+    private boolean nextTerm() throws IOException {
+      if ((thisTerm = termsEnum.next()) == null)
+        return false;
+      indexedCell = grid.readCell(thisTerm, indexedCell);
+      return true;
     }
 
     /** Used for {@link VNode#children}. */
@@ -322,23 +319,21 @@ public abstract class AbstractVisitingPr
     protected abstract DocIdSet finish() throws IOException;
 
     /**
-     * Visit an indexed non-leaf cell returned from
-     * {@link #findSubCellsToVisit(org.apache.lucene.spatial.prefix.tree.Cell)}
-     * that is also found in the index.
-     * It will also be called by the default implementation of
-     * {@link #visitScanned(org.apache.lucene.spatial.prefix.tree.Cell)} for
-     * cells at the bottom detail level.
+     * Visit an indexed non-leaf cell. The presence of a prefix cell implies
+     * there are leaf cells at further levels. The cell passed should have it's
+     * {@link org.apache.lucene.spatial.prefix.tree.Cell#getShapeRel()} set
+     * relative to the filtered shape.
      *
      * @param cell An intersecting cell; not a leaf.
-     * @return true to descend to more levels. It is an error to return true
-     * if cell.level == detailLevel
+     * @return true to descend to more levels.
      */
-    protected abstract boolean visit(Cell cell) throws IOException;
+    protected abstract boolean visitPrefix(Cell cell) throws IOException;
 
     /**
      * Called when an indexed leaf cell is found. An
-     * indexed leaf cell means associated documents generally won't be found at
-     * further detail levels.
+     * indexed leaf cell usually means associated documents won't be found at
+     * further detail levels.  However, if a document has
+     * multiple overlapping shapes at different resolutions, then this isn't true.
      */
     protected abstract void visitLeaf(Cell cell) throws IOException;
 
@@ -347,14 +342,16 @@ public abstract class AbstractVisitingPr
      * might not even intersect the query shape, so be sure to check for that.
      * The default implementation will check that and if passes then call
      * {@link #visitLeaf(org.apache.lucene.spatial.prefix.tree.Cell)} or
-     * {@link #visit(org.apache.lucene.spatial.prefix.tree.Cell)}.
+     * {@link #visitPrefix(org.apache.lucene.spatial.prefix.tree.Cell)}.
      */
     protected void visitScanned(Cell cell) throws IOException {
-      if (queryShape.relate(cell.getShape()).intersects()) {
+      final SpatialRelation relate = cell.getShape().relate(queryShape);
+      if (relate.intersects()) {
+        cell.setShapeRel(relate);//just being pedantic
         if (cell.isLeaf()) {
           visitLeaf(cell);
         } else {
-          visit(cell);
+          visitPrefix(cell);
         }
       }
     }

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java Tue Mar 31 05:22:40 2015
@@ -17,11 +17,14 @@ package org.apache.lucene.spatial.prefix
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.Arrays;
+
 import com.spatial4j.core.shape.Shape;
 import com.spatial4j.core.shape.SpatialRelation;
-
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.spatial.prefix.tree.Cell;
@@ -32,9 +35,6 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.SentinelIntSet;
 
-import java.io.IOException;
-import java.util.Arrays;
-
 /**
  * Finds docs where its indexed shape {@link org.apache.lucene.spatial.query.SpatialOperation#Contains
  * CONTAINS} the query shape. For use on {@link RecursivePrefixTreeStrategy}.
@@ -43,11 +43,6 @@ import java.util.Arrays;
  */
 public class ContainsPrefixTreeFilter extends AbstractPrefixTreeFilter {
 
-  /*
-  Future optimizations:
-    Instead of seekExact, use seekCeil with some leap-frogging, like Intersects does.
-  */
-
   /**
    * If the spatial data for a document is comprised of multiple overlapping or adjacent parts,
    * it might fail to match a query shape when doing the CONTAINS predicate when the sum of
@@ -77,8 +72,8 @@ public class ContainsPrefixTreeFilter ex
   @Override
   public String toString(String field) {
     return "ContainsPrefixTreeFilter(" +
-        // TODO: print something about the shape?
         "fieldName=" + fieldName + "," +
+        "queryShape=" + queryShape + "," +
         "detailLevel=" + detailLevel + "," +
         "multiOverlappingIndexedShapes=" + multiOverlappingIndexedShapes +
         ")";
@@ -93,21 +88,21 @@ public class ContainsPrefixTreeFilter ex
 
     public ContainsVisitor(LeafReaderContext context, Bits acceptDocs) throws IOException {
       super(context, acceptDocs);
+      if (termsEnum != null) {
+        nextTerm();//advance to first
+      }
     }
 
-    //The reused value of cell.getTokenBytesNoLeaf which is always then seek()'ed to. It's used in assertions too.
-    BytesRef termBytes = new BytesRef();//no leaf
-    Cell nextCell;//see getLeafDocs
+    BytesRef seekTerm = new BytesRef();//temp; see seek()
+    BytesRef thisTerm;//current term in termsEnum
+    Cell indexedCell;//the cell wrapper around thisTerm
 
     /** This is the primary algorithm; recursive.  Returns null if finds none. */
     private SmallDocSet visit(Cell cell, Bits acceptContains) throws IOException {
 
-      if (termsEnum == null)//signals all done
+      if (thisTerm == null)//signals all done
         return null;
 
-      // Leaf docs match all query shape
-      SmallDocSet leafDocs = getLeafDocs(cell, acceptContains);
-
       // Get the AND of all child results (into combinedSubResults)
       SmallDocSet combinedSubResults = null;
       //   Optimization: use null subCellsFilter when we know cell is within the query shape.
@@ -119,68 +114,105 @@ public class ContainsPrefixTreeFilter ex
       CellIterator subCells = cell.getNextLevelCells(subCellsFilter);
       while (subCells.hasNext()) {
         Cell subCell = subCells.next();
-        if (!seekExact(subCell))
+        if (!seek(subCell)) {
           combinedSubResults = null;
-        else if (subCell.getLevel() == detailLevel)
+        } else if (subCell.getLevel() == detailLevel) {
           combinedSubResults = getDocs(subCell, acceptContains);
-        else if (!multiOverlappingIndexedShapes &&
-            subCell.getShapeRel() == SpatialRelation.WITHIN)
+        } else if (!multiOverlappingIndexedShapes &&
+            subCell.getShapeRel() == SpatialRelation.WITHIN) {
           combinedSubResults = getLeafDocs(subCell, acceptContains);
-        else
-          combinedSubResults = visit(subCell, acceptContains); //recursion
+        } else {
+          //OR the leaf docs with all child results
+          SmallDocSet leafDocs = getLeafDocs(subCell, acceptContains);
+          SmallDocSet subDocs = visit(subCell, acceptContains); //recursion
+          combinedSubResults = union(leafDocs, subDocs);
+        }
 
         if (combinedSubResults == null)
           break;
         acceptContains = combinedSubResults;//has the 'AND' effect on next iteration
       }
 
-      // Result: OR the leaf docs with AND of all child results
-      if (combinedSubResults != null) {
-        if (leafDocs == null)
-          return combinedSubResults;
-        return leafDocs.union(combinedSubResults);//union is 'or'
-      }
-      return leafDocs;
+      return combinedSubResults;
     }
 
-    private boolean seekExact(Cell cell) throws IOException {
-      assert cell.getTokenBytesNoLeaf(null).compareTo(termBytes) > 0;
-      if (termsEnum == null)
+    private boolean seek(Cell cell) throws IOException {
+      if (thisTerm == null)
         return false;
-      termBytes = cell.getTokenBytesNoLeaf(termBytes);
-      assert assertCloneTermBytes(); //assertions look at termBytes later on
-      return termsEnum.seekExact(termBytes);
-    }
-
-    private boolean assertCloneTermBytes() {
-      termBytes = BytesRef.deepCopyOf(termBytes);
-      return true;
+      final int compare = indexedCell.compareToNoLeaf(cell);
+      if (compare > 0) {
+        return false;//leap-frog effect
+      } else if (compare == 0) {
+        return true; // already there!
+      } else {//compare > 0
+        //seek!
+        seekTerm = cell.getTokenBytesNoLeaf(seekTerm);
+        final TermsEnum.SeekStatus seekStatus = termsEnum.seekCeil(seekTerm);
+        if (seekStatus == TermsEnum.SeekStatus.END) {
+          thisTerm = null;//all done
+          return false;
+        }
+        thisTerm = termsEnum.term();
+        indexedCell = grid.readCell(thisTerm, indexedCell);
+        if (seekStatus == TermsEnum.SeekStatus.FOUND) {
+          return true;
+        }
+        return indexedCell.isLeaf() && indexedCell.compareToNoLeaf(cell) == 0;
+      }
     }
 
+    /** Get prefix & leaf docs at this cell. */
     private SmallDocSet getDocs(Cell cell, Bits acceptContains) throws IOException {
-      assert cell.getTokenBytesNoLeaf(null).equals(termBytes);
-
-      return collectDocs(acceptContains);
+      assert indexedCell.compareToNoLeaf(cell) == 0;
+      //called when we've reached detailLevel.
+      if (indexedCell.isLeaf()) {//only a leaf
+        SmallDocSet result = collectDocs(acceptContains);
+        nextTerm();
+        return result;
+      } else {
+        SmallDocSet docsAtPrefix = collectDocs(acceptContains);
+        if (!nextTerm()) {
+          return docsAtPrefix;
+        }
+        //collect leaf too
+        if (indexedCell.isLeaf() && indexedCell.compareToNoLeaf(cell) == 0) {
+          SmallDocSet docsAtLeaf = collectDocs(acceptContains);
+          nextTerm();
+          return union(docsAtPrefix, docsAtLeaf);
+        } else {
+          return docsAtPrefix;
+        }
+      }
     }
 
     /** Gets docs on the leaf of the given cell, _if_ there is a leaf cell, otherwise null. */
     private SmallDocSet getLeafDocs(Cell cell, Bits acceptContains) throws IOException {
-      assert cell.getTokenBytesNoLeaf(null).equals(termBytes);
-
-      if (termsEnum == null)
-        return null;
-      BytesRef nextTerm = termsEnum.next();
-      if (nextTerm == null) {
-        termsEnum = null;//signals all done
-        return null;
+      assert indexedCell.compareToNoLeaf(cell) == 0;
+      //Advance past prefix if we're at a prefix; return null if no leaf
+      if (!indexedCell.isLeaf()) {
+        if (!nextTerm() || !indexedCell.isLeaf() || indexedCell.getLevel() != cell.getLevel()) {
+          return null;
+        }
       }
-      nextCell = grid.readCell(nextTerm, nextCell);
-      assert cell.isPrefixOf(nextCell);
-      if (nextCell.getLevel() == cell.getLevel() && nextCell.isLeaf()) {
-        return collectDocs(acceptContains);
-      } else {
-        return null;
+      SmallDocSet result = collectDocs(acceptContains);
+      nextTerm();
+      return result;
+    }
+
+    private boolean nextTerm() throws IOException {
+      if ((thisTerm = termsEnum.next()) == null)
+        return false;
+      indexedCell = grid.readCell(thisTerm, indexedCell);
+      return true;
+    }
+
+    private SmallDocSet union(SmallDocSet aSet, SmallDocSet bSet) {
+      if (bSet != null) {
+        if (aSet == null)
+          return bSet;
+        return aSet.union(bSet);//union is 'or'
       }
+      return aSet;
     }
 
     private SmallDocSet collectDocs(Bits acceptContains) throws IOException {

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/IntersectsPrefixTreeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/IntersectsPrefixTreeFilter.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/IntersectsPrefixTreeFilter.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/IntersectsPrefixTreeFilter.java Tue Mar 31 05:22:40 2015
@@ -39,8 +39,8 @@ public class IntersectsPrefixTreeFilter
 
   public IntersectsPrefixTreeFilter(Shape queryShape, String fieldName,
                                     SpatialPrefixTree grid, int detailLevel,
-                                    int prefixGridScanLevel, boolean hasIndexedLeaves) {
-    super(queryShape, fieldName, grid, detailLevel, prefixGridScanLevel, hasIndexedLeaves);
+                                    int prefixGridScanLevel) {
+    super(queryShape, fieldName, grid, detailLevel, prefixGridScanLevel);
   }
 
   @Override
@@ -68,7 +68,7 @@ public class IntersectsPrefixTreeFilter
       }
 
       @Override
-      protected boolean visit(Cell cell) throws IOException {
+      protected boolean visitPrefix(Cell cell) throws IOException {
         if (cell.getShapeRel() == SpatialRelation.WITHIN || cell.getLevel() == detailLevel) {
           collectDocs(results);
           return false;
@@ -87,11 +87,10 @@ public class IntersectsPrefixTreeFilter
   @Override
   public String toString(String field) {
     return "IntersectsPrefixTreeFilter(" +
-        // TODO: print something about the shape?
         "fieldName=" + fieldName + "," +
+        "queryShape=" + queryShape + "," +
         "detailLevel=" + detailLevel + "," +
-        "prefixGridScanLevel=" + prefixGridScanLevel + "," +
-        "hasIndexedLeaves=" + hasIndexedLeaves +
+        "prefixGridScanLevel=" + prefixGridScanLevel +
         ")";
   }
 

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java Tue Mar 31 05:22:40 2015
@@ -19,14 +19,13 @@ package org.apache.lucene.spatial.prefix
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Iterator;
 import java.util.Map;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
 import com.spatial4j.core.shape.Point;
 import com.spatial4j.core.shape.Shape;
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.Filter;
@@ -59,11 +58,9 @@ public class NumberRangePrefixTreeStrate
   }
 
   @Override
-  public Field[] createIndexableFields(Shape shape) {
+  protected Iterator<Cell> createCellIteratorToIndex(Shape shape, int detailLevel, Iterator<Cell> reuse) {
     //levels doesn't actually matter; NumberRange based Shapes have their own "level".
-    TokenStream tokenStream = createTokenStream(shape, grid.getMaxLevels());
-    Field field = new Field(getFieldName(), tokenStream, FIELD_TYPE);
-    return new Field[]{field};
+    return super.createCellIteratorToIndex(shape, grid.getMaxLevels(), reuse);
   }
 
   /** Unsupported. */

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java Tue Mar 31 05:22:40 2015
@@ -24,7 +24,8 @@ import org.apache.lucene.spatial.util.Sh
 import org.apache.lucene.util.BytesRef;
 
 /**
- * Implementation of {@link ShapeFieldCacheProvider} designed for {@link PrefixTreeStrategy}s that index points.
+ * Implementation of {@link ShapeFieldCacheProvider} designed for {@link PrefixTreeStrategy}s that index points
+ * (AND ONLY POINTS!).
  *
  * @lucene.internal
  */
@@ -40,8 +41,8 @@ public class PointPrefixTreeFieldCachePr
 
   @Override
   protected Point readShape(BytesRef term) {
-    scanCell = grid.readCell(term, scanCell);;
-    if (scanCell.getLevel() == grid.getMaxLevels() && !scanCell.isLeaf())//points are never flagged as leaf
+    scanCell = grid.readCell(term, scanCell);
+    if (scanCell.getLevel() == grid.getMaxLevels())
       return scanCell.getShape().getCenter();
     return null;
   }

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeFacetCounter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeFacetCounter.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeFacetCounter.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeFacetCounter.java Tue Mar 31 05:22:40 2015
@@ -20,9 +20,9 @@ package org.apache.lucene.spatial.prefix
 import java.io.IOException;
 
 import com.spatial4j.core.shape.Shape;
-import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Filter;
@@ -33,6 +33,17 @@ import org.apache.lucene.util.SparseFixe
 
 /**
  * Computes facets on cells for {@link org.apache.lucene.spatial.prefix.PrefixTreeStrategy}.
+ * <p>
+ * <em>NOTE:</em> If for a given document and a given field using
+ * {@link org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy}
+ * multiple values are indexed (i.e. multi-valued) and at least one of them is a non-point, then there is a possibility
+ * of double-counting the document in the facet results.  Since each shape is independently turned into grid cells at
+ * a resolution chosen by the shape's size, it's possible they will be indexed at different resolutions.  This means
+ * the document could be present in BOTH the postings for a cell in both its prefix and leaf variants.  To avoid this,
+ * use a single valued field with a {@link com.spatial4j.core.shape.ShapeCollection} (or WKT equivalent).  Or
+ * calculate a suitable level/distErr to index both and call
+ * {@link org.apache.lucene.spatial.prefix.PrefixTreeStrategy#createIndexableFields(com.spatial4j.core.shape.Shape, int)}
+ * with the same value for all shapes for a given document/field.
  *
  * @lucene.experimental
  */
@@ -114,8 +125,7 @@ public class PrefixTreeFacetCounter {
 
     //AbstractVisitingPrefixTreeFilter is a Lucene Filter.  We don't need a filter; we use it for its great prefix-tree
     // traversal code.  TODO consider refactoring if/when it makes sense (more use cases than this)
-    new AbstractVisitingPrefixTreeFilter(queryShape, strategy.getFieldName(), tree, facetLevel, scanLevel,
-        !strategy.isPointsOnly()) {
+    new AbstractVisitingPrefixTreeFilter(queryShape, strategy.getFieldName(), tree, facetLevel, scanLevel) {
       
       @Override
       public String toString(String field) {
@@ -139,7 +149,7 @@ public class PrefixTreeFacetCounter {
           }
 
           @Override
-          protected boolean visit(Cell cell) throws IOException {
+          protected boolean visitPrefix(Cell cell) throws IOException {
             // At facetLevel...
             if (cell.getLevel() == facetLevel) {
               // Count docs

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java Tue Mar 31 05:22:40 2015
@@ -24,7 +24,6 @@ import java.util.concurrent.ConcurrentHa
 
 import com.spatial4j.core.shape.Point;
 import com.spatial4j.core.shape.Shape;
-import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.index.IndexOptions;
@@ -127,8 +126,8 @@ public abstract class PrefixTreeStrategy
     return pointsOnly;
   }
 
-  /** True if only indexed points shall be supported. There are no "leafs" in such a case.  See
-   *  {@link org.apache.lucene.spatial.prefix.IntersectsPrefixTreeFilter#hasIndexedLeaves}. */
+  /** True if only indexed points shall be supported. There are no "leafs" in such a case, except those
+   * at maximum precision. */
   public void setPointsOnly(boolean pointsOnly) {
     this.pointsOnly = pointsOnly;
   }
@@ -142,23 +141,33 @@ public abstract class PrefixTreeStrategy
   /**
    * Turns {@link SpatialPrefixTree#getTreeCellIterator(Shape, int)} into a
    * {@link org.apache.lucene.analysis.TokenStream}.
-   * {@code simplifyIndexedCells} is an optional hint affecting non-point shapes: it will
-   * simply/aggregate sets of complete leaves in a cell to its parent, resulting in ~20-25%
-   * fewer cells. It will likely be removed in the future.
    */
   public Field[] createIndexableFields(Shape shape, double distErr) {
     int detailLevel = grid.getLevelForDistance(distErr);
-    TokenStream tokenStream = createTokenStream(shape, detailLevel);
+    return createIndexableFields(shape, detailLevel);
+  }
+
+  public Field[] createIndexableFields(Shape shape, int detailLevel) {
+    //TODO re-use TokenStream LUCENE-5776: Subclass Field, put cell iterator there, override tokenStream()
+    Iterator<Cell> cells = createCellIteratorToIndex(shape, detailLevel, null);
+    CellToBytesRefIterator cellToBytesRefIterator = newCellToBytesRefIterator();
+    cellToBytesRefIterator.reset(cells);
+    BytesRefIteratorTokenStream tokenStream = new BytesRefIteratorTokenStream();
+    tokenStream.setBytesRefIterator(cellToBytesRefIterator);
     Field field = new Field(getFieldName(), tokenStream, FIELD_TYPE);
     return new Field[]{field};
   }
 
-  protected TokenStream createTokenStream(Shape shape, int detailLevel) {
+  protected CellToBytesRefIterator newCellToBytesRefIterator() {
+    //subclasses could return one that never emits leaves, or does both, or who knows.
+    return new CellToBytesRefIterator();
+  }
+
+  protected Iterator<Cell> createCellIteratorToIndex(Shape shape, int detailLevel, Iterator<Cell> reuse) {
     if (pointsOnly && !(shape instanceof Point)) {
       throw new IllegalArgumentException("pointsOnly is true yet a " + shape.getClass() + " is given for indexing");
     }
-    Iterator<Cell> cells = grid.getTreeCellIterator(shape, detailLevel);
-    return new CellTokenStream().setCells(cells);
+    return grid.getTreeCellIterator(shape, detailLevel);//TODO should take a re-use iterator
   }
 
   /* Indexed, tokenized, not stored. */

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java Tue Mar 31 05:22:40 2015
@@ -18,11 +18,11 @@ package org.apache.lucene.spatial.prefix
  */
 
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 
 import com.spatial4j.core.shape.Point;
 import com.spatial4j.core.shape.Shape;
-import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.spatial.prefix.tree.Cell;
 import org.apache.lucene.spatial.prefix.tree.CellIterator;
@@ -116,13 +116,13 @@ public class RecursivePrefixTreeStrategy
   }
 
   @Override
-  protected TokenStream createTokenStream(Shape shape, int detailLevel) {
+  protected Iterator<Cell> createCellIteratorToIndex(Shape shape, int detailLevel, Iterator<Cell> reuse) {
     if (shape instanceof Point || !pruneLeafyBranches)
-      return super.createTokenStream(shape, detailLevel);
+      return super.createCellIteratorToIndex(shape, detailLevel, reuse);
 
     List<Cell> cells = new ArrayList<>(4096);
     recursiveTraverseAndPrune(grid.getWorldCell(), shape, detailLevel, cells);
-    return new CellTokenStream().setCells(cells.iterator());
+    return cells.iterator();
   }
 
   /** Returns true if cell was added as a leaf. If it wasn't it recursively descends. */
@@ -174,10 +174,10 @@ public class RecursivePrefixTreeStrategy
 
     if (op == SpatialOperation.Intersects) {
       return new IntersectsPrefixTreeFilter(
-          shape, getFieldName(), grid, detailLevel, prefixGridScanLevel, !pointsOnly);
+          shape, getFieldName(), grid, detailLevel, prefixGridScanLevel);
     } else if (op == SpatialOperation.IsWithin) {
       return new WithinPrefixTreeFilter(
-          shape, getFieldName(), grid, detailLevel, prefixGridScanLevel, !pointsOnly,
+          shape, getFieldName(), grid, detailLevel, prefixGridScanLevel,
           -1);//-1 flag is slower but ensures correct results
     } else if (op == SpatialOperation.Contains) {
       return new ContainsPrefixTreeFilter(shape, getFieldName(), grid, detailLevel,

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java Tue Mar 31 05:22:40 2015
@@ -20,6 +20,8 @@ package org.apache.lucene.spatial.prefix
 import java.util.ArrayList;
 import java.util.List;
 
+import com.spatial4j.core.shape.Point;
+import com.spatial4j.core.shape.Shape;
 import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.QueryWrapperFilter;
@@ -32,9 +34,6 @@ import org.apache.lucene.spatial.query.U
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Shape;
-
 /**
  * A basic implementation of {@link PrefixTreeStrategy} using a large
  * {@link TermsQuery} of all the cells from
@@ -56,6 +55,20 @@ public class TermQueryPrefixTreeStrategy
   }
 
   @Override
+  protected CellToBytesRefIterator newCellToBytesRefIterator() {
+    //Ensure we don't have leaves, as this strategy doesn't handle them.
+    return new CellToBytesRefIterator() {
+      @Override
+      public BytesRef next() {
+        if (!cellIter.hasNext()) {
+          return null;
+        }
+        return cellIter.next().getTokenBytesNoLeaf(bytesRef);
+      }
+    };
+  }
+
+  @Override
   public Filter makeFilter(SpatialArgs args) {
     final SpatialOperation op = args.getOperation();
     if (op != SpatialOperation.Intersects)
@@ -92,7 +105,8 @@ public class TermQueryPrefixTreeStrategy
     for (BytesRef byteRef : terms) {
       byteRef.bytes = masterBytes.bytes();
     }
-    //unfortunately TermsFilter will needlessly sort & dedupe
+    //unfortunately TermsQuery will needlessly sort & dedupe
+    //TODO an automatonQuery might be faster?
     return new QueryWrapperFilter(new TermsQuery(getFieldName(), terms));
   }
 

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/WithinPrefixTreeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/WithinPrefixTreeFilter.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/WithinPrefixTreeFilter.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/WithinPrefixTreeFilter.java Tue Mar 31 05:22:40 2015
@@ -60,15 +60,15 @@ public class WithinPrefixTreeFilter exte
   private final Shape bufferedQueryShape;//if null then the whole world
 
   /**
-   * See {@link AbstractVisitingPrefixTreeFilter#AbstractVisitingPrefixTreeFilter(com.spatial4j.core.shape.Shape, String, org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree, int, int, boolean)}.
+   * See {@link AbstractVisitingPrefixTreeFilter#AbstractVisitingPrefixTreeFilter(com.spatial4j.core.shape.Shape, String, org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree, int, int)}.
    * {@code queryBuffer} is the (minimum) distance beyond the query shape edge
    * where non-matching documents are looked for so they can be excluded. If
    * -1 is used then the whole world is examined (a good default for correctness).
    */
   public WithinPrefixTreeFilter(Shape queryShape, String fieldName, SpatialPrefixTree grid,
-                                int detailLevel, int prefixGridScanLevel, boolean hasIndexedLeaves,
+                                int detailLevel, int prefixGridScanLevel,
                                 double queryBuffer) {
-    super(queryShape, fieldName, grid, detailLevel, prefixGridScanLevel, hasIndexedLeaves);
+    super(queryShape, fieldName, grid, detailLevel, prefixGridScanLevel);
     this.bufferedQueryShape = queryBuffer == -1 ? null : bufferShape(queryShape, queryBuffer);
   }
 
@@ -94,11 +94,10 @@ public class WithinPrefixTreeFilter exte
   @Override
   public String toString(String field) {
     return "WithinPrefixTreeFilter(" +
-             // TODO: print something about the shape?
              "fieldName=" + fieldName + "," +
+             "queryShape=" + queryShape + "," +
              "detailLevel=" + detailLevel + "," +
-             "prefixGridScanLevel=" + prefixGridScanLevel + "," +
-             "hasIndexedLeaves=" + hasIndexedLeaves +
+             "prefixGridScanLevel=" + prefixGridScanLevel +
            ")";
   }
 
@@ -152,7 +151,6 @@ public class WithinPrefixTreeFilter exte
     return new VisitorTemplate(context, acceptDocs) {
       private FixedBitSet inside;
       private FixedBitSet outside;
-      private SpatialRelation visitRelation;
 
       @Override
       protected void start() {
@@ -173,32 +171,26 @@ public class WithinPrefixTreeFilter exte
       }
 
       @Override
-      protected boolean visit(Cell cell) throws IOException {
+      protected boolean visitPrefix(Cell cell) throws IOException {
         //cell.relate is based on the bufferedQueryShape; we need to examine what
         // the relation is against the queryShape
-        visitRelation = cell.getShape().relate(queryShape);
-        if (visitRelation == SpatialRelation.WITHIN) {
+        SpatialRelation visitRelation = cell.getShape().relate(queryShape);
+        if (cell.getLevel() == detailLevel) {
+          collectDocs(visitRelation.intersects() ? inside : outside);
+          return false;
+        } else if (visitRelation == SpatialRelation.WITHIN) {
           collectDocs(inside);
           return false;
         } else if (visitRelation == SpatialRelation.DISJOINT) {
           collectDocs(outside);
           return false;
-        } else if (cell.getLevel() == detailLevel) {
-          collectDocs(inside);
-          return false;
         }
         return true;
       }
 
       @Override
       protected void visitLeaf(Cell cell) throws IOException {
-        //visitRelation is declared as a field, populated by visit() so we don't recompute it.
-        // We have a specialized visitScanned() which doesn't call this. If we didn't, we would
-        // not be able to assume visitRelation is from a prior visit() call since in scanning,
-        // parent cells aren't visited.
-        assert detailLevel != cell.getLevel();
-        assert visitRelation == cell.getShape().relate(queryShape);
-        if (allCellsIntersectQuery(cell, visitRelation))
+        if (allCellsIntersectQuery(cell))
           collectDocs(inside);
         else
           collectDocs(outside);
@@ -207,9 +199,8 @@ public class WithinPrefixTreeFilter exte
       /** Returns true if the provided cell, and all its sub-cells down to
        * detailLevel all intersect the queryShape.
        */
-      private boolean allCellsIntersectQuery(Cell cell, SpatialRelation relate/*cell to query*/) {
-        if (relate == null)
-          relate = cell.getShape().relate(queryShape);
+      private boolean allCellsIntersectQuery(Cell cell) {
+        SpatialRelation relate = cell.getShape().relate(queryShape);
         if (cell.getLevel() == detailLevel)
           return relate.intersects();
         if (relate == SpatialRelation.WITHIN)
@@ -221,7 +212,7 @@ public class WithinPrefixTreeFilter exte
         CellIterator subCells = cell.getNextLevelCells(null);
         while (subCells.hasNext()) {
           Cell subCell = subCells.next();
-          if (!allCellsIntersectQuery(subCell, null))//recursion
+          if (!allCellsIntersectQuery(subCell))//recursion
             return false;
         }
         return true;
@@ -229,12 +220,12 @@ public class WithinPrefixTreeFilter exte
 
       @Override
       protected void visitScanned(Cell cell) throws IOException {
-        //slightly optimize over default impl; required for our 'visitRelation' field re-use above
-        if (allCellsIntersectQuery(cell, null)) {
-          collectDocs(inside);
-        } else {
-          collectDocs(outside);
-        }
+        visitLeaf(cell);//collects as we want, even if not a leaf
+//        if (cell.isLeaf()) {
+//          visitLeaf(cell);
+//        } else {
+//          visitPrefix(cell);
+//        }
       }
 
     }.getDocIdSet();

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java Tue Mar 31 05:22:40 2015
@@ -51,7 +51,6 @@ public interface Cell {
    * Some cells are flagged as leaves, which are indexed as such. A leaf cell is either within some
    * shape or it both intersects and the cell is at an accuracy threshold such that no smaller cells
    * for the shape will be represented.
-   * further cells with this prefix for the shape (always true at maxLevels).
    */
   boolean isLeaf();
 
@@ -62,14 +61,14 @@ public interface Cell {
   void setLeaf();
 
   /**
-   * Returns the bytes for this cell, with a leaf byte if this is a leaf cell.
+   * Returns the bytes for this cell, with a leaf byte <em>if this is a leaf cell</em>.
    * The result param is used to save object allocation, though its bytes aren't used.
    * @param result where the result goes, or null to create new
    */
   BytesRef getTokenBytesWithLeaf(BytesRef result);
 
   /**
-   * Returns the bytes for this cell, without leaf set. The bytes should sort before
+   * Returns the bytes for this cell, without a leaf set. The bytes should sort before
    * {@link #getTokenBytesWithLeaf(org.apache.lucene.util.BytesRef)}.
    * The result param is used to save object allocation, though its bytes aren't used.
    * @param result where the result goes, or null to create new

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java Tue Mar 31 05:22:40 2015
@@ -17,6 +17,10 @@ package org.apache.lucene.spatial.prefix
  * limitations under the License.
  */
 
+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;
@@ -24,10 +28,6 @@ import com.spatial4j.core.shape.Rectangl
 import com.spatial4j.core.shape.Shape;
 import org.apache.lucene.util.BytesRef;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
 /**
  * A {@link SpatialPrefixTree} based on
  * <a href="http://en.wikipedia.org/wiki/Geohash">Geohashes</a>.
@@ -104,7 +104,7 @@ public class GeohashPrefixTree extends L
     GhCell(String geohash) {
       super(stringToBytesPlus1(geohash), 0, geohash.length());
       this.geohash = geohash;
-      if (isLeaf())
+      if (isLeaf() && getLevel() < getMaxLevels())//we don't have a leaf byte at max levels (an opt)
         this.geohash = geohash.substring(0, geohash.length() - 1);
     }
 
@@ -116,6 +116,9 @@ public class GeohashPrefixTree extends L
     protected GeohashPrefixTree getGrid() { return GeohashPrefixTree.this; }
 
     @Override
+    protected int getMaxLevels() { return maxLevels; }
+
+    @Override
     protected void readCell(BytesRef bytesRef) {
       super.readCell(bytesRef);
       geohash = null;

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java Tue Mar 31 05:22:40 2015
@@ -17,14 +17,14 @@ package org.apache.lucene.spatial.prefix
  * limitations under the License.
  */
 
+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;
 
-import java.util.Collection;
-
 /** 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
@@ -64,7 +64,7 @@ public abstract class LegacyCell impleme
     shape = null;
     this.bytes = bytes.bytes;
     this.b_off = bytes.offset;
-    this.b_len = bytes.length;
+    this.b_len = (short) bytes.length;
     readLeafAdjust();
   }
 
@@ -72,6 +72,8 @@ public abstract class LegacyCell impleme
     isLeaf = (b_len > 0 && bytes[b_off + b_len - 1] == LEAF_BYTE);
     if (isLeaf)
       b_len--;
+    if (getLevel() == getMaxLevels())
+      isLeaf = true;
   }
 
 //  @Override
@@ -88,6 +90,8 @@ public abstract class LegacyCell impleme
 
   protected abstract SpatialPrefixTree getGrid();
 
+  protected abstract int getMaxLevels();
+
   @Override
   public SpatialRelation getShapeRel() {
     return shapeRel;
@@ -111,7 +115,7 @@ public abstract class LegacyCell impleme
   @Override
   public BytesRef getTokenBytesWithLeaf(BytesRef result) {
     result = getTokenBytesNoLeaf(result);
-    if (!isLeaf)
+    if (!isLeaf || getLevel() == getMaxLevels())
       return result;
     if (result.bytes.length < result.offset + result.length + 1) {
       assert false : "Not supposed to happen; performance bug";

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyPrefixTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyPrefixTree.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyPrefixTree.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyPrefixTree.java Tue Mar 31 05:22:40 2015
@@ -17,14 +17,14 @@ package org.apache.lucene.spatial.prefix
  * limitations under the License.
  */
 
+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;
 
-import java.util.Arrays;
-
 /** 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 {
@@ -69,7 +69,7 @@ abstract class LegacyPrefixTree extends
     // 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.isLeaf() && cell instanceof LegacyCell;
+    assert cell instanceof LegacyCell;
     BytesRef fullBytes = cell.getTokenBytesNoLeaf(null);
     //fill in reverse order to be sorted
     Cell[] cells = new Cell[detailLevel];

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java Tue Mar 31 05:22:40 2015
@@ -17,14 +17,6 @@ package org.apache.lucene.spatial.prefix
  * limitations under the License.
  */
 
-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;
-
 import java.io.PrintStream;
 import java.text.NumberFormat;
 import java.util.ArrayList;
@@ -33,6 +25,13 @@ 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
@@ -231,6 +230,9 @@ public class QuadPrefixTree extends Lega
     protected QuadPrefixTree getGrid() { return QuadPrefixTree.this; }
 
     @Override
+    protected int getMaxLevels() { return maxLevels; }
+
+    @Override
     protected Collection<Cell> getSubCells() {
       BytesRef source = getTokenBytesNoLeaf(null);
 

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java Tue Mar 31 05:22:40 2015
@@ -54,7 +54,7 @@ public abstract class SpatialPrefixTreeF
       instance = new QuadPrefixTree.Factory();
     else {
       try {
-        Class c = classLoader.loadClass(cname);
+        Class<?> c = classLoader.loadClass(cname);
         instance = (SpatialPrefixTreeFactory) c.newInstance();
       } catch (Exception e) {
         throw new RuntimeException(e);

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCache.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCache.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCache.java Tue Mar 31 05:22:40 2015
@@ -32,10 +32,10 @@ import java.util.List;
  * @lucene.internal
  */
 public class ShapeFieldCache<T extends Shape> {
-  private List<T>[] cache;
-  public int defaultLength;
+  private final List<T>[] cache;
+  public final int defaultLength;
 
-  @SuppressWarnings({"unchecked"})
+  @SuppressWarnings({"unchecked", "rawtypes"})
   public ShapeFieldCache( int length, int defaultLength ) {
     cache = new List[length];
     this.defaultLength= defaultLength;

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialTestCase.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialTestCase.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialTestCase.java Tue Mar 31 05:22:40 2015
@@ -29,6 +29,8 @@ import com.spatial4j.core.context.Spatia
 import com.spatial4j.core.distance.DistanceUtils;
 import com.spatial4j.core.shape.Point;
 import com.spatial4j.core.shape.Rectangle;
+
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.DirectoryReader;
@@ -46,8 +48,6 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
 import org.apache.lucene.util.TestUtil;
-import org.junit.After;
-import org.junit.Before;
 
 import static com.carrotsearch.randomizedtesting.RandomizedTest.randomDouble;
 import static com.carrotsearch.randomizedtesting.RandomizedTest.randomGaussian;
@@ -63,6 +63,7 @@ public abstract class SpatialTestCase ex
   private DirectoryReader indexReader;
   protected RandomIndexWriter indexWriter;
   private Directory directory;
+  private Analyzer analyzer;
   protected IndexSearcher indexSearcher;
 
   protected SpatialContext ctx;//subclass must initialize
@@ -70,7 +71,6 @@ public abstract class SpatialTestCase ex
   protected Map<String,Type> uninvertMap = new HashMap<>();
   
   @Override
-  @Before
   public void setUp() throws Exception {
     super.setUp();
     // TODO: change this module to index docvalues instead of uninverting
@@ -80,13 +80,14 @@ public abstract class SpatialTestCase ex
 
     directory = newDirectory();
     final Random random = random();
-    indexWriter = new RandomIndexWriter(random,directory, newIndexWriterConfig(random));
+    analyzer = new MockAnalyzer(random);
+    indexWriter = new RandomIndexWriter(random,directory, newIWConfig(random, analyzer));
     indexReader = UninvertingReader.wrap(indexWriter.getReader(), uninvertMap);
     indexSearcher = newSearcher(indexReader);
   }
 
-  protected IndexWriterConfig newIndexWriterConfig(Random random) {
-    final IndexWriterConfig indexWriterConfig = LuceneTestCase.newIndexWriterConfig(random, new MockAnalyzer(random));
+  protected IndexWriterConfig newIWConfig(Random random, Analyzer analyzer) {
+    final IndexWriterConfig indexWriterConfig = LuceneTestCase.newIndexWriterConfig(random, analyzer);
     //TODO can we randomly choose a doc-values supported format?
     if (needsDocValues())
       indexWriterConfig.setCodec( TestUtil.getDefaultCodec());
@@ -98,10 +99,8 @@ public abstract class SpatialTestCase ex
   }
 
   @Override
-  @After
   public void tearDown() throws Exception {
-    indexWriter.close();
-    IOUtils.close(indexReader,directory);
+    IOUtils.close(indexWriter, indexReader, analyzer, directory);
     super.tearDown();
   }
 

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/DateNRStrategyTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/DateNRStrategyTest.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/DateNRStrategyTest.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/DateNRStrategyTest.java Tue Mar 31 05:22:40 2015
@@ -28,6 +28,7 @@ import org.apache.lucene.spatial.query.S
 import org.junit.Before;
 import org.junit.Test;
 
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomBoolean;
 import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween;
 
 public class DateNRStrategyTest extends RandomSpatialOpStrategyTestCase {
@@ -42,7 +43,17 @@ public class DateNRStrategyTest extends
   public void setUp() throws Exception {
     super.setUp();
     tree = DateRangePrefixTree.INSTANCE;
-    strategy = new NumberRangePrefixTreeStrategy(tree, "dateRange");
+    if (randomBoolean()) {
+      strategy = new NumberRangePrefixTreeStrategy(tree, "dateRange");
+    } else {
+      //Test the format that existed <= Lucene 5.0
+      strategy = new NumberRangePrefixTreeStrategy(tree, "dateRange") {
+        @Override
+        protected CellToBytesRefIterator newCellToBytesRefIterator() {
+          return new CellToBytesRefIterator50();
+        }
+      };
+    }
     Calendar tmpCal = tree.newCal();
     int randomCalWindowField = randomIntBetween(1, Calendar.ZONE_OFFSET - 1);//we're not allowed to add zone offset
     tmpCal.add(randomCalWindowField, 2_000);

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/HeatmapFacetCounterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/HeatmapFacetCounterTest.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/HeatmapFacetCounterTest.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/HeatmapFacetCounterTest.java Tue Mar 31 05:22:40 2015
@@ -31,7 +31,6 @@ import com.spatial4j.core.shape.Rectangl
 import com.spatial4j.core.shape.Shape;
 import com.spatial4j.core.shape.SpatialRelation;
 import com.spatial4j.core.shape.impl.RectangleImpl;
-
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.FilteredQuery;
 import org.apache.lucene.search.MatchAllDocsQuery;
@@ -235,7 +234,7 @@ public class HeatmapFacetCounterTest ext
     // we use IntersectsPrefixTreeFilter directly so that we can specify the level to go to exactly.
     RecursivePrefixTreeStrategy strategy = (RecursivePrefixTreeStrategy) this.strategy;
     Filter filter = new IntersectsPrefixTreeFilter(
-        pt, strategy.getFieldName(), grid, facetLevel, grid.getMaxLevels(), !strategy.isPointsOnly());
+        pt, strategy.getFieldName(), grid, facetLevel, grid.getMaxLevels());
     final TotalHitCountCollector collector = new TotalHitCountCollector();
     indexSearcher.search(new FilteredQuery(new MatchAllDocsQuery(), filter), collector);
     cellsValidated++;

Modified: lucene/dev/branches/lucene6271/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpFuzzyPrefixTreeTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpFuzzyPrefixTreeTest.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpFuzzyPrefixTreeTest.java (original)
+++ lucene/dev/branches/lucene6271/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpFuzzyPrefixTreeTest.java Tue Mar 31 05:22:40 2015
@@ -67,7 +67,7 @@ public class RandomSpatialOpFuzzyPrefixT
 
   static final int ITERATIONS = 10;
 
-  private SpatialPrefixTree grid;
+  protected SpatialPrefixTree grid;
   private SpatialContext ctx2D;
 
   public void setupGrid(int maxLevels) throws IOException {
@@ -108,7 +108,7 @@ public class RandomSpatialOpFuzzyPrefixT
     if (maxLevels == -1)
       maxLevels = randomIntBetween(1, 8);//max 64k cells (4^8), also 256*256
     this.grid = new QuadPrefixTree(ctx, maxLevels);
-    this.strategy = new RecursivePrefixTreeStrategy(grid, getClass().getSimpleName());
+    this.strategy = newRPT();
   }
 
   public void setupGeohashGrid(int maxLevels) {
@@ -117,7 +117,11 @@ public class RandomSpatialOpFuzzyPrefixT
     if (maxLevels == -1)
       maxLevels = randomIntBetween(1, 3);//max 16k cells (32^3)
     this.grid = new GeohashPrefixTree(ctx, maxLevels);
-    this.strategy = new RecursivePrefixTreeStrategy(grid, getClass().getSimpleName());
+    this.strategy = newRPT();
+  }
+
+  protected RecursivePrefixTreeStrategy newRPT() {
+    return new RecursivePrefixTreeStrategy(this.grid, getClass().getSimpleName());
   }
 
   @Test
@@ -226,6 +230,7 @@ public class RandomSpatialOpFuzzyPrefixT
     return doc;
   }
 
+  @SuppressWarnings("fallthrough")
   private void doTest(final SpatialOperation operation) throws IOException {
     //first show that when there's no data, a query will result in no results
     {

Modified: lucene/dev/branches/lucene6271/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java (original)
+++ lucene/dev/branches/lucene6271/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java Tue Mar 31 05:22:40 2015
@@ -49,7 +49,7 @@ public class FSTUtil {
     public final FST.Arc<T> fstNode;
 
     /** Output of the path so far: */
-    T output;
+    public final T output;
 
     /** Input of the path so far: */
     public final IntsRefBuilder input;

Modified: lucene/dev/branches/lucene6271/lucene/suggest/src/test/org/apache/lucene/search/spell/TestDirectSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/suggest/src/test/org/apache/lucene/search/spell/TestDirectSpellChecker.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/suggest/src/test/org/apache/lucene/search/spell/TestDirectSpellChecker.java (original)
+++ lucene/dev/branches/lucene6271/lucene/suggest/src/test/org/apache/lucene/search/spell/TestDirectSpellChecker.java Tue Mar 31 05:22:40 2015
@@ -17,6 +17,7 @@ package org.apache.lucene.search.spell;
  * limitations under the License.
  */
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.document.Document;
@@ -26,6 +27,7 @@ import org.apache.lucene.index.RandomInd
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.English;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 
 public class TestDirectSpellChecker extends LuceneTestCase {
@@ -33,8 +35,8 @@ public class TestDirectSpellChecker exte
   public void testInternalLevenshteinDistance() throws Exception {
     DirectSpellChecker spellchecker = new DirectSpellChecker();
     Directory dir = newDirectory();
-    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, 
-        new MockAnalyzer(random(), MockTokenizer.KEYWORD, true));
+    Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.KEYWORD, true);
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, analyzer);
 
     String[] termsToAdd = { "metanoia", "metanoian", "metanoiai", "metanoias", "metanoi𐑍" };
     for (int i = 0; i < termsToAdd.length; i++) {
@@ -55,16 +57,15 @@ public class TestDirectSpellChecker exte
       assertTrue(word.score==sd.getDistance(misspelled, word.string));
     }
     
-    ir.close();
-    writer.close();
-    dir.close();
+    IOUtils.close(ir, writer, dir, analyzer);
   }
+  
   public void testSimpleExamples() throws Exception {
     DirectSpellChecker spellChecker = new DirectSpellChecker();
     spellChecker.setMinQueryLength(0);
     Directory dir = newDirectory();
-    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, 
-        new MockAnalyzer(random(), MockTokenizer.SIMPLE, true));
+    Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, analyzer);
 
     for (int i = 0; i < 20; i++) {
       Document doc = new Document();
@@ -121,15 +122,13 @@ public class TestDirectSpellChecker exte
     assertTrue(similar.length > 0); 
     assertEquals("thousand", similar[0].string);
 
-    ir.close();
-    writer.close();
-    dir.close();
+    IOUtils.close(ir, writer, dir, analyzer);
   }
   
   public void testOptions() throws Exception {
     Directory dir = newDirectory();
-    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, 
-        new MockAnalyzer(random(), MockTokenizer.SIMPLE, true));
+    Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, analyzer);
 
     Document doc = new Document();
     doc.add(newTextField("text", "foobar", Field.Store.NO));
@@ -187,16 +186,14 @@ public class TestDirectSpellChecker exte
         SuggestMode.SUGGEST_ALWAYS);
     assertEquals(2, similar.length);
 
-    ir.close();
-    writer.close();
-    dir.close();
+    IOUtils.close(ir, writer, dir, analyzer);;
   }
   
   public void testBogusField() throws Exception {
     DirectSpellChecker spellChecker = new DirectSpellChecker();
     Directory dir = newDirectory();
-    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, 
-        new MockAnalyzer(random(), MockTokenizer.SIMPLE, true));
+    Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, analyzer);
 
     for (int i = 0; i < 20; i++) {
       Document doc = new Document();
@@ -210,17 +207,16 @@ public class TestDirectSpellChecker exte
         "bogusFieldBogusField", "fvie"), 2, ir,
         SuggestMode.SUGGEST_WHEN_NOT_IN_INDEX);
     assertEquals(0, similar.length);
-    ir.close();
-    writer.close();
-    dir.close();
+    
+    IOUtils.close(ir, writer, dir, analyzer);
   }
   
   // simple test that transpositions work, we suggest five for fvie with ed=1
   public void testTransposition() throws Exception {
     DirectSpellChecker spellChecker = new DirectSpellChecker();
     Directory dir = newDirectory();
-    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, 
-        new MockAnalyzer(random(), MockTokenizer.SIMPLE, true));
+    Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, analyzer);
 
     for (int i = 0; i < 20; i++) {
       Document doc = new Document();
@@ -235,17 +231,16 @@ public class TestDirectSpellChecker exte
         SuggestMode.SUGGEST_WHEN_NOT_IN_INDEX);
     assertEquals(1, similar.length);
     assertEquals("five", similar[0].string);
-    ir.close();
-    writer.close();
-    dir.close();
+    
+    IOUtils.close(ir, writer, dir, analyzer);
   }
   
   // simple test that transpositions work, we suggest seventeen for seevntene with ed=2
   public void testTransposition2() throws Exception {
     DirectSpellChecker spellChecker = new DirectSpellChecker();
     Directory dir = newDirectory();
-    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, 
-        new MockAnalyzer(random(), MockTokenizer.SIMPLE, true));
+    Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, analyzer);
 
     for (int i = 0; i < 20; i++) {
       Document doc = new Document();
@@ -260,8 +255,7 @@ public class TestDirectSpellChecker exte
         SuggestMode.SUGGEST_WHEN_NOT_IN_INDEX);
     assertEquals(1, similar.length);
     assertEquals("seventeen", similar[0].string);
-    ir.close();
-    writer.close();
-    dir.close();
+    
+    IOUtils.close(ir, writer, dir, analyzer);
   }
 }

Modified: lucene/dev/branches/lucene6271/lucene/suggest/src/test/org/apache/lucene/search/spell/TestLuceneDictionary.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/suggest/src/test/org/apache/lucene/search/spell/TestLuceneDictionary.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/suggest/src/test/org/apache/lucene/search/spell/TestLuceneDictionary.java (original)
+++ lucene/dev/branches/lucene6271/lucene/suggest/src/test/org/apache/lucene/search/spell/TestLuceneDictionary.java Tue Mar 31 05:22:40 2015
@@ -19,6 +19,7 @@ package org.apache.lucene.search.spell;
 
 import java.io.IOException;
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.document.Document;
@@ -39,7 +40,7 @@ import org.apache.lucene.util.LuceneTest
 public class TestLuceneDictionary extends LuceneTestCase {
 
   private Directory store;
-
+  private Analyzer analyzer;
   private IndexReader indexReader = null;
   private LuceneDictionary ld;
   private BytesRefIterator it;
@@ -49,7 +50,8 @@ public class TestLuceneDictionary extend
   public void setUp() throws Exception {
     super.setUp();
     store = newDirectory();
-    IndexWriter writer = new IndexWriter(store, newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false)));
+    analyzer = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
+    IndexWriter writer = new IndexWriter(store, newIndexWriterConfig(analyzer));
 
     Document doc;
 
@@ -82,6 +84,7 @@ public class TestLuceneDictionary extend
     if (indexReader != null)
       indexReader.close();
     store.close();
+    analyzer.close();
     super.tearDown();
   }
   

Modified: lucene/dev/branches/lucene6271/lucene/suggest/src/test/org/apache/lucene/search/spell/TestSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/suggest/src/test/org/apache/lucene/search/spell/TestSpellChecker.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/suggest/src/test/org/apache/lucene/search/spell/TestSpellChecker.java (original)
+++ lucene/dev/branches/lucene6271/lucene/suggest/src/test/org/apache/lucene/search/spell/TestSpellChecker.java Tue Mar 31 05:22:40 2015
@@ -27,6 +27,7 @@ import java.util.concurrent.ExecutorServ
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -47,6 +48,7 @@ import org.apache.lucene.util.NamedThrea
 public class TestSpellChecker extends LuceneTestCase {
   private SpellCheckerMock spellChecker;
   private Directory userindex, spellindex;
+  private Analyzer analyzer;
   private List<IndexSearcher> searchers;
 
   @Override
@@ -55,7 +57,8 @@ public class TestSpellChecker extends Lu
     
     //create a user index
     userindex = newDirectory();
-    IndexWriter writer = new IndexWriter(userindex, new IndexWriterConfig(new MockAnalyzer(random())));
+    analyzer = new MockAnalyzer(random());
+    IndexWriter writer = new IndexWriter(userindex, new IndexWriterConfig(analyzer));
 
     for (int i = 0; i < 1000; i++) {
       Document doc = new Document();
@@ -99,6 +102,7 @@ public class TestSpellChecker extends Lu
     if (!spellChecker.isClosed())
       spellChecker.close();
     spellindex.close();
+    analyzer.close();
     super.tearDown();
   }