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 2015/04/23 06:17:22 UTC

svn commit: r1675539 - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/benchmark/ lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/ lucene/spatial/ lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ lucene/spatial/src/java...

Author: dsmiley
Date: Thu Apr 23 04:17:22 2015
New Revision: 1675539

URL: http://svn.apache.org/r1675539
Log:
LUCENE-6422: New spatial PackedQuadPrefixTree. Thanks Nick!

Added:
    lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java
      - copied, changed from r1675538, lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java
Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/lucene/   (props changed)
    lucene/dev/branches/branch_5x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/lucene/benchmark/   (props changed)
    lucene/dev/branches/branch_5x/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/SpatialDocMaker.java
    lucene/dev/branches/branch_5x/lucene/spatial/   (props changed)
    lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java
    lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/CellIterator.java
    lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java
    lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java
    lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java
    lucene/dev/branches/branch_5x/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java
    lucene/dev/branches/branch_5x/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpFuzzyPrefixTreeTest.java
    lucene/dev/branches/branch_5x/solr/   (props changed)
    lucene/dev/branches/branch_5x/solr/core/   (props changed)
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/SpatialRecursivePrefixTreeFieldType.java
    lucene/dev/branches/branch_5x/solr/core/src/test-files/solr/collection1/conf/schema-spatial.xml
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java

Modified: lucene/dev/branches/branch_5x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/CHANGES.txt?rev=1675539&r1=1675538&r2=1675539&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/lucene/CHANGES.txt Thu Apr 23 04:17:22 2015
@@ -44,6 +44,11 @@ New Features
 * LUCENE-6423: New LimitTokenOffsetFilter that limits tokens to those before
   a configured maximum start offset. (David Smiley)
 
+* LUCENE-6422: New spatial PackedQuadPrefixTree, a generally more efficient
+
  choice than QuadPrefixTree, especially for high precision shapes.
+
  When used, you should typically disable RPT's pruneLeafyBranches option.

+  (Nick Knize, David Smiley)
+
 Optimizations
 
 * LUCENE-6379: IndexWriter.deleteDocuments(Query...) now detects if

Modified: lucene/dev/branches/branch_5x/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/SpatialDocMaker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/SpatialDocMaker.java?rev=1675539&r1=1675538&r2=1675539&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/SpatialDocMaker.java (original)
+++ lucene/dev/branches/branch_5x/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/SpatialDocMaker.java Thu Apr 23 04:17:22 2015
@@ -33,6 +33,7 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.spatial.SpatialStrategy;
 import org.apache.lucene.spatial.composite.CompositeSpatialStrategy;
 import org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy;
+import org.apache.lucene.spatial.prefix.tree.PackedQuadPrefixTree;
 import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
 import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTreeFactory;
 import org.apache.lucene.spatial.serialized.SerializedDVStrategy;
@@ -111,7 +112,13 @@ public class SpatialDocMaker extends Doc
 
     RecursivePrefixTreeStrategy strategy = new RecursivePrefixTreeStrategy(grid, spatialField);
     strategy.setPointsOnly(config.get("spatial.docPointsOnly", false));
-    strategy.setPruneLeafyBranches(config.get("spatial.pruneLeafyBranches", true));
+    final boolean pruneLeafyBranches = config.get("spatial.pruneLeafyBranches", true);
+    if (grid instanceof PackedQuadPrefixTree) {
+      ((PackedQuadPrefixTree) grid).setPruneLeafyBranches(pruneLeafyBranches);
+      strategy.setPruneLeafyBranches(false);//always leave it to packed grid, even though it isn't the same
+    } else {
+      strategy.setPruneLeafyBranches(pruneLeafyBranches);
+    }
 
     int prefixGridScanLevel = config.get("query.spatial.prefixGridScanLevel", -4);
     if (prefixGridScanLevel < 0)

Modified: lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java?rev=1675539&r1=1675538&r2=1675539&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java (original)
+++ lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java Thu Apr 23 04:17:22 2015
@@ -92,9 +92,14 @@ public class RecursivePrefixTreeStrategy
     return pruneLeafyBranches;
   }
 
-  /** An optional hint affecting non-point shapes: it will
-   * simplify/aggregate sets of complete leaves in a cell to its parent, resulting in ~20-25%
-   * fewer indexed cells. However, it will likely be removed in the future. (default=true)
+  /**
+   * An optional hint affecting non-point shapes: it will
+   * prune away a complete set sibling leaves to their parent (recursively), resulting in ~20-50%
+   * fewer indexed cells, and consequently that much less disk and that much faster indexing.
+   * So if it's a quad tree and all 4 sub-cells are there marked as a leaf, then they will be
+   * removed (pruned) and the parent is marked as a leaf instead.  This occurs recursively on up.  Unfortunately, the
+   * current implementation will buffer all cells to do this, so consider disabling for high precision (low distErrPct)
+   * shapes. (default=true)
    */
   public void setPruneLeafyBranches(boolean pruneLeafyBranches) {
     this.pruneLeafyBranches = pruneLeafyBranches;

Modified: lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/CellIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/CellIterator.java?rev=1675539&r1=1675538&r2=1675539&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/CellIterator.java (original)
+++ lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/CellIterator.java Thu Apr 23 04:17:22 2015
@@ -65,7 +65,7 @@ public abstract class CellIterator imple
   }
 
   @Override
-  public final Cell next() {
+  public Cell next() {
     if (nextCell == null) {
       if (!hasNext())
         throw new NoSuchElementException();

Modified: lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java?rev=1675539&r1=1675538&r2=1675539&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java (original)
+++ lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java Thu Apr 23 04:17:22 2015
@@ -36,9 +36,9 @@ public abstract class LegacyCell impleme
   private static final byte LEAF_BYTE = '+';//NOTE: must sort before letters & numbers
 
   //Arguably we could simply use a BytesRef, using an extra Object.
-  private byte[] bytes;//generally bigger to potentially hold a leaf
-  private int b_off;
-  private int b_len;//doesn't reflect leaf; same as getLevel()
+  protected byte[] bytes;//generally bigger to potentially hold a leaf
+  protected int b_off;
+  protected int b_len;//doesn't reflect leaf; same as getLevel()
 
   protected boolean isLeaf;
 
@@ -68,7 +68,7 @@ public abstract class LegacyCell impleme
     readLeafAdjust();
   }
 
-  private void readLeafAdjust() {
+  protected void readLeafAdjust() {
     isLeaf = (b_len > 0 && bytes[b_off + b_len - 1] == LEAF_BYTE);
     if (isLeaf)
       b_len--;
@@ -76,18 +76,6 @@ public abstract class LegacyCell impleme
       isLeaf = true;
   }
 
-//  @Override
-//  public void copyFrom(Cell source) {
-//    LegacyCell src = (LegacyCell) source;
-//    shapeRel = src.shapeRel;
-//    shape = src.shape;
-//    isLeaf = src.isLeaf;
-//    //we don't actually copy the bytes because in LegacyCell the bytes aren't modified. (leaf byte doesn't count)
-//    bytes = src.bytes;
-//    b_off = src.b_off;
-//    b_len = src.b_len;
-//  }
-
   protected abstract SpatialPrefixTree getGrid();
 
   protected abstract int getMaxLevels();
@@ -214,7 +202,7 @@ public abstract class LegacyCell impleme
 
   /** Copied from {@link BytesRef#compareTo(BytesRef)}.
    * This is to avoid creating a BytesRef. */
-  private static int compare(byte[] aBytes, int aUpto, int a_length, byte[] bBytes, int bUpto, int b_length) {
+  protected static int compare(byte[] aBytes, int aUpto, int a_length, byte[] bBytes, int bUpto, int b_length) {
     final int aStop = aUpto + Math.min(a_length, b_length);
     while(aUpto < aStop) {
       int aByte = aBytes[aUpto++] & 0xff;

Copied: lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java (from r1675538, lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java?p2=lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java&p1=lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java&r1=1675538&r2=1675539&rev=1675539&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java (original)
+++ lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java Thu Apr 23 04:17:22 2015
@@ -251,12 +251,17 @@ public class PackedQuadPrefixTree extend
       PackedQuadCell b = (PackedQuadCell) fromCell;
       final long thisTerm = (((0x1L)&term) == 0x1L) ? term-1 : term;
       final long fromTerm = (((0x1L)&b.term) == 0x1L) ? b.term-1 : b.term;
-      final int result = Long.compareUnsigned(thisTerm, fromTerm);
+      final int result = Long_compareUnsigned(thisTerm, fromTerm);
       assert Math.signum(result)
           == Math.signum(compare(longToByteArray(thisTerm), 0, 8, longToByteArray(fromTerm), 0, 8)); // TODO remove
       return result;
     }
 
+    //For Java 1.7; this is in 1.8
+    private int Long_compareUnsigned(long x, long y) {
+      return Long.compare(x + Long.MIN_VALUE, y + Long.MIN_VALUE);
+    }
+
     @Override
     public int getLevel() {
       int l = (int)((term >>> 1)&0x1FL);

Modified: lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java?rev=1675539&r1=1675538&r2=1675539&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java (original)
+++ lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java Thu Apr 23 04:17:22 2015
@@ -62,14 +62,14 @@ public class QuadPrefixTree extends Lega
   public static final int MAX_LEVELS_POSSIBLE = 50;//not really sure how big this should be
 
   public static final int DEFAULT_MAX_LEVELS = 12;
-  private final double xmin;
-  private final double xmax;
-  private final double ymin;
-  private final double ymax;
-  private final double xmid;
-  private final double ymid;
+  protected final double xmin;
+  protected final double xmax;
+  protected final double ymin;
+  protected final double ymax;
+  protected final double xmid;
+  protected final double ymid;
 
-  private final double gridW;
+  protected final double gridW;
   public final double gridH;
 
   final double[] levelW;
@@ -178,7 +178,7 @@ public class QuadPrefixTree extends Lega
     // if we actually use the range property in the query, this could be useful
   }
 
-  private void checkBattenberg(
+  protected void checkBattenberg(
       char c,
       double cx,
       double cy,
@@ -215,7 +215,7 @@ public class QuadPrefixTree extends Lega
     str.length = strlen;
   }
 
-  private class QuadCell extends LegacyCell {
+  protected class QuadCell extends LegacyCell {
 
     QuadCell(byte[] bytes, int off, int len) {
       super(bytes, off, len);
@@ -244,7 +244,7 @@ public class QuadPrefixTree extends Lega
       return cells;
     }
 
-    private BytesRef concat(BytesRef source, byte b) {
+    protected BytesRef concat(BytesRef source, byte b) {
       //+2 for new char + potential leaf
       final byte[] buffer = Arrays.copyOfRange(source.bytes, source.offset, source.offset + source.length + 2);
       BytesRef target = new BytesRef(buffer);
@@ -270,7 +270,7 @@ public class QuadPrefixTree extends Lega
       return shape;
     }
 
-    private Rectangle makeShape() {
+    protected Rectangle makeShape() {
       BytesRef token = getTokenBytesNoLeaf(null);
       double xmin = QuadPrefixTree.this.xmin;
       double ymin = QuadPrefixTree.this.ymin;

Modified: lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java?rev=1675539&r1=1675538&r2=1675539&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java (original)
+++ lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java Thu Apr 23 04:17:22 2015
@@ -17,11 +17,11 @@
 
 package org.apache.lucene.spatial.prefix.tree;
 
+import java.util.Map;
+
 import com.spatial4j.core.context.SpatialContext;
 import com.spatial4j.core.distance.DistanceUtils;
 
-import java.util.Map;
-
 /**
  * Abstract Factory for creating {@link SpatialPrefixTree} instances with useful
  * defaults and passed on configurations defined in a Map.
@@ -52,6 +52,8 @@ public abstract class SpatialPrefixTreeF
       instance = new GeohashPrefixTree.Factory();
     else if ("quad".equalsIgnoreCase(cname))
       instance = new QuadPrefixTree.Factory();
+    else if ("packedQuad".equalsIgnoreCase(cname))
+      instance = new PackedQuadPrefixTree.Factory();
     else {
       try {
         Class<?> c = classLoader.loadClass(cname);

Modified: lucene/dev/branches/branch_5x/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java?rev=1675539&r1=1675538&r2=1675539&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java (original)
+++ lucene/dev/branches/branch_5x/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java Thu Apr 23 04:17:22 2015
@@ -22,22 +22,23 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
+import com.carrotsearch.randomizedtesting.annotations.Name;
+import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
+import com.spatial4j.core.context.SpatialContext;
+import com.spatial4j.core.shape.Point;
+import com.spatial4j.core.shape.Shape;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.spatial.bbox.BBoxStrategy;
 import org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy;
 import org.apache.lucene.spatial.prefix.TermQueryPrefixTreeStrategy;
 import org.apache.lucene.spatial.prefix.tree.GeohashPrefixTree;
+import org.apache.lucene.spatial.prefix.tree.PackedQuadPrefixTree;
 import org.apache.lucene.spatial.prefix.tree.QuadPrefixTree;
 import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
 import org.apache.lucene.spatial.serialized.SerializedDVStrategy;
 import org.apache.lucene.spatial.vector.PointVectorStrategy;
 import org.junit.Test;
-import com.carrotsearch.randomizedtesting.annotations.Name;
-import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Shape;
 
 public class DistanceStrategyTest extends StrategyTestCase {
 
@@ -57,6 +58,10 @@ public class DistanceStrategyTest extend
     strategy = new TermQueryPrefixTreeStrategy(grid, "termquery_geohash");
     ctorArgs.add(new Object[]{new Param(strategy)});
 
+    grid = new PackedQuadPrefixTree(ctx,25);
+    strategy = new RecursivePrefixTreeStrategy(grid, "recursive_packedquad");
+    ctorArgs.add(new Object[]{new Param(strategy)});
+
     strategy = new PointVectorStrategy(ctx, "pointvector");
     ctorArgs.add(new Object[]{new Param(strategy)});
 

Modified: lucene/dev/branches/branch_5x/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpFuzzyPrefixTreeTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpFuzzyPrefixTreeTest.java?rev=1675539&r1=1675538&r2=1675539&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpFuzzyPrefixTreeTest.java (original)
+++ lucene/dev/branches/branch_5x/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpFuzzyPrefixTreeTest.java Thu Apr 23 04:17:22 2015
@@ -47,6 +47,7 @@ import org.apache.lucene.spatial.Strateg
 import org.apache.lucene.spatial.prefix.tree.Cell;
 import org.apache.lucene.spatial.prefix.tree.CellIterator;
 import org.apache.lucene.spatial.prefix.tree.GeohashPrefixTree;
+import org.apache.lucene.spatial.prefix.tree.PackedQuadPrefixTree;
 import org.apache.lucene.spatial.prefix.tree.QuadPrefixTree;
 import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
 import org.apache.lucene.spatial.query.SpatialArgs;
@@ -72,14 +73,17 @@ public class RandomSpatialOpFuzzyPrefixT
 
   public void setupGrid(int maxLevels) throws IOException {
     if (randomBoolean())
-      setupQuadGrid(maxLevels);
+      setupQuadGrid(maxLevels, randomBoolean());
     else
       setupGeohashGrid(maxLevels);
     setupCtx2D(ctx);
 
-    //((PrefixTreeStrategy) strategy).setDistErrPct(0);//fully precise to grid
-
+    // set prune independently on strategy & grid randomly; should work
     ((RecursivePrefixTreeStrategy)strategy).setPruneLeafyBranches(randomBoolean());
+    if (this.grid instanceof PackedQuadPrefixTree) {
+      ((PackedQuadPrefixTree) this.grid).setPruneLeafyBranches(randomBoolean());
+    }
+
     if (maxLevels == -1 && rarely()) {
       ((PrefixTreeStrategy) strategy).setPointsOnly(true);
     }
@@ -97,7 +101,7 @@ public class RandomSpatialOpFuzzyPrefixT
     ctx2D = ctxFactory.newSpatialContext();
   }
 
-  private void setupQuadGrid(int maxLevels) {
+  private void setupQuadGrid(int maxLevels, boolean packedQuadPrefixTree) {
     //non-geospatial makes this test a little easier (in gridSnap), and using boundary values 2^X raises
     // the prospect of edge conditions we want to test, plus makes for simpler numbers (no decimals).
     SpatialContextFactory factory = new SpatialContextFactory();
@@ -107,7 +111,11 @@ public class RandomSpatialOpFuzzyPrefixT
     //A fairly shallow grid, and default 2.5% distErrPct
     if (maxLevels == -1)
       maxLevels = randomIntBetween(1, 8);//max 64k cells (4^8), also 256*256
-    this.grid = new QuadPrefixTree(ctx, maxLevels);
+    if (packedQuadPrefixTree) {
+      this.grid = new PackedQuadPrefixTree(ctx, maxLevels);
+    } else {
+      this.grid = new QuadPrefixTree(ctx, maxLevels);
+    }
     this.strategy = newRPT();
   }
 
@@ -148,7 +156,7 @@ public class RandomSpatialOpFuzzyPrefixT
   /** See LUCENE-5062, {@link ContainsPrefixTreeFilter#multiOverlappingIndexedShapes}. */
   @Test
   public void testContainsPairOverlap() throws IOException {
-    setupQuadGrid(3);
+    setupQuadGrid(3, randomBoolean());
     adoc("0", new ShapePair(ctx.makeRectangle(0, 33, -128, 128), ctx.makeRectangle(33, 128, -128, 128), true));
     commit();
     Query query = strategy.makeQuery(new SpatialArgs(SpatialOperation.Contains,
@@ -159,7 +167,7 @@ public class RandomSpatialOpFuzzyPrefixT
 
   @Test
   public void testWithinDisjointParts() throws IOException {
-    setupQuadGrid(7);
+    setupQuadGrid(7, randomBoolean());
     //one shape comprised of two parts, quite separated apart
     adoc("0", new ShapePair(ctx.makeRectangle(0, 10, -120, -100), ctx.makeRectangle(220, 240, 110, 125), false));
     commit();
@@ -173,7 +181,7 @@ public class RandomSpatialOpFuzzyPrefixT
 
   @Test /** LUCENE-4916 */
   public void testWithinLeafApproxRule() throws IOException {
-    setupQuadGrid(2);//4x4 grid
+    setupQuadGrid(2, randomBoolean());//4x4 grid
     //indexed shape will simplify to entire right half (2 top cells)
     adoc("0", ctx.makeRectangle(192, 204, -128, 128));
     commit();

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/SpatialRecursivePrefixTreeFieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/SpatialRecursivePrefixTreeFieldType.java?rev=1675539&r1=1675538&r2=1675539&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/SpatialRecursivePrefixTreeFieldType.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/SpatialRecursivePrefixTreeFieldType.java Thu Apr 23 04:17:22 2015
@@ -17,10 +17,11 @@ package org.apache.solr.schema;
  * limitations under the License.
  */
 
-import org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy;
-
 import java.util.Map;
 
+import org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy;
+import org.apache.lucene.spatial.prefix.tree.PackedQuadPrefixTree;
+
 /**
  * @see RecursivePrefixTreeStrategy
  * @lucene.experimental
@@ -45,6 +46,11 @@ public class SpatialRecursivePrefixTreeF
     RecursivePrefixTreeStrategy strategy = new RecursivePrefixTreeStrategy(grid, fieldName);
     if (prefixGridScanLevel != null)
       strategy.setPrefixGridScanLevel(prefixGridScanLevel);
+    if (grid instanceof PackedQuadPrefixTree) {
+      // This grid has a (usually) better prune leafy branch implementation
+      ((PackedQuadPrefixTree) grid).setPruneLeafyBranches(true);
+      strategy.setPruneLeafyBranches(false);
+    }
     return strategy;
   }
 }

Modified: lucene/dev/branches/branch_5x/solr/core/src/test-files/solr/collection1/conf/schema-spatial.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test-files/solr/collection1/conf/schema-spatial.xml?rev=1675539&r1=1675538&r2=1675539&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test-files/solr/collection1/conf/schema-spatial.xml (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test-files/solr/collection1/conf/schema-spatial.xml Thu Apr 23 04:17:22 2015
@@ -35,6 +35,9 @@
     <fieldType name="srpt_quad"   class="solr.SpatialRecursivePrefixTreeFieldType"
               prefixTree="quad" distanceUnits="degrees"
         />
+    <fieldType name="srpt_packedquad"   class="solr.SpatialRecursivePrefixTreeFieldType"
+              prefixTree="packedQuad" distanceUnits="degrees"
+        />
     <fieldType name="srpt_100km"   class="solr.SpatialRecursivePrefixTreeFieldType"
               maxDistErr="100" distanceUnits="kilometers"
         />
@@ -58,6 +61,7 @@
 
     <field name="srpt_geohash" type="srpt_geohash" multiValued="true" />
     <field name="srpt_quad" type="srpt_quad" multiValued="true" />
+    <field name="srpt_packedquad" type="srpt_packedquad" multiValued="true" />
     <field name="stqpt_geohash" type="stqpt_geohash" multiValued="true" />
     <field name="pointvector" type="pointvector" />
     <field name="bbox" type="bbox" />

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java?rev=1675539&r1=1675538&r2=1675539&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java Thu Apr 23 04:17:22 2015
@@ -26,7 +26,6 @@ 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.spatial.bbox.BBoxStrategy;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
@@ -54,7 +53,7 @@ public class TestSolr4Spatial extends So
   @ParametersFactory
   public static Iterable<Object[]> parameters() {
     return Arrays.asList(new Object[][]{
-        {"srpt_geohash"}, {"srpt_quad"}, {"stqpt_geohash"}, {"pointvector"}, {"bbox"}
+        {"srpt_geohash"}, {"srpt_quad"}, {"srpt_packedquad"}, {"stqpt_geohash"}, {"pointvector"}, {"bbox"}
     });
   }