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 2017/05/26 20:00:53 UTC

lucene-solr:master: LUCENE-7845: RPT query by point (or simple date interval) optimization

Repository: lucene-solr
Updated Branches:
  refs/heads/master b23aab548 -> d4f87b4a3


LUCENE-7845: RPT query by point (or simple date interval) optimization


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/d4f87b4a
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/d4f87b4a
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/d4f87b4a

Branch: refs/heads/master
Commit: d4f87b4a36ca50c4361d7ec4e0858b18d9eaebe8
Parents: b23aab5
Author: David Smiley <ds...@apache.org>
Authored: Fri May 26 16:00:28 2017 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Fri May 26 16:00:28 2017 -0400

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 ++
 .../prefix/NumberRangePrefixTreeStrategy.java   | 24 ++++++++---
 .../spatial/prefix/PrefixTreeStrategy.java      | 20 +++++++--
 .../prefix/RecursivePrefixTreeStrategy.java     | 41 +++++++++++++++++--
 .../spatial/prefix/DateNRStrategyTest.java      | 43 +++++++++++---------
 5 files changed, 100 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d4f87b4a/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index a45c11a..0dfc709 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -88,6 +88,10 @@ Optimizations
   values using different numbers of bits per value if this proves to save
   storage. (Adrien Grand)
 
+* LUCENE-7845: Enhance spatial-extras RecursivePrefixTreeStrategy queries when the
+  query is a point (for 2D) or a is a simple date interval (e.g. 1 month).  When
+  the strategy is marked as pointsOnly, the results is a TermQuery. (David Smiley)
+
 Other
 
 * LUCENE-7328: Remove LegacyNumericEncoding from GeoPointField. (Nick Knize)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d4f87b4a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java
index c727c0d..8367644 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java
@@ -18,18 +18,17 @@ 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 org.locationtech.spatial4j.shape.Point;
-import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.spatial.prefix.tree.Cell;
 import org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree;
 import org.apache.lucene.util.Bits;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
 
 import static org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape;
 
@@ -57,9 +56,22 @@ public class NumberRangePrefixTreeStrategy extends RecursivePrefixTreeStrategy {
   }
 
   @Override
-  protected Iterator<Cell> createCellIteratorToIndex(Shape shape, int detailLevel, Iterator<Cell> reuse) {
-    //levels doesn't actually matter; NumberRange based Shapes have their own "level".
-    return super.createCellIteratorToIndex(shape, grid.getMaxLevels(), reuse);
+  protected boolean isPointShape(Shape shape) {
+    if (shape instanceof NumberRangePrefixTree.UnitNRShape) {
+      return ((NumberRangePrefixTree.UnitNRShape)shape).getLevel() == grid.getMaxLevels();
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  protected boolean isGridAlignedShape(Shape shape) {
+    // any UnitNRShape other than the world is a single cell/term
+    if (shape instanceof NumberRangePrefixTree.UnitNRShape) {
+      return ((NumberRangePrefixTree.UnitNRShape)shape).getLevel() > 0;
+    } else {
+      return false;
+    }
   }
 
   /** Unsupported. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d4f87b4a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
index e9f43fd..43851c7 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
@@ -21,8 +21,6 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
-import org.locationtech.spatial4j.shape.Point;
-import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.index.IndexOptions;
@@ -34,6 +32,10 @@ import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
 import org.apache.lucene.spatial.query.SpatialArgs;
 import org.apache.lucene.spatial.util.ShapeFieldCacheDistanceValueSource;
 import org.apache.lucene.util.Bits;
+import org.locationtech.spatial4j.shape.Circle;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
 
 /**
  * An abstract SpatialStrategy based on {@link SpatialPrefixTree}. The two
@@ -163,7 +165,7 @@ public abstract class PrefixTreeStrategy extends SpatialStrategy {
   }
 
   protected Iterator<Cell> createCellIteratorToIndex(Shape shape, int detailLevel, Iterator<Cell> reuse) {
-    if (pointsOnly && !(shape instanceof Point)) {
+    if (pointsOnly && !isPointShape(shape)) {
       throw new IllegalArgumentException("pointsOnly is true yet a " + shape.getClass() + " is given for indexing");
     }
     return grid.getTreeCellIterator(shape, detailLevel);//TODO should take a re-use iterator
@@ -205,4 +207,16 @@ public abstract class PrefixTreeStrategy extends SpatialStrategy {
                                    Shape inputShape, final int facetLevel, int maxCells) throws IOException {
     return HeatmapFacetCounter.calcFacets(this, context, topAcceptDocs, inputShape, facetLevel, maxCells);
   }
+
+  protected boolean isPointShape(Shape shape) {
+    if (shape instanceof Point) {
+      return true;
+    } else if (shape instanceof Circle) {
+      return ((Circle) shape).getRadius() == 0.0;
+    } else if (shape instanceof Rectangle) {
+      Rectangle rect = (Rectangle) shape;
+      return rect.getWidth() == 0.0 && rect.getHeight() == 0.0;
+    }
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d4f87b4a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java
index d3d1626..7c79200 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java
@@ -20,9 +20,9 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
-import org.locationtech.spatial4j.shape.Point;
-import org.locationtech.spatial4j.shape.Shape;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.spatial.prefix.tree.Cell;
 import org.apache.lucene.spatial.prefix.tree.CellIterator;
 import org.apache.lucene.spatial.prefix.tree.LegacyCell;
@@ -30,6 +30,7 @@ import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
 import org.apache.lucene.spatial.query.SpatialArgs;
 import org.apache.lucene.spatial.query.SpatialOperation;
 import org.apache.lucene.spatial.query.UnsupportedSpatialOperation;
+import org.locationtech.spatial4j.shape.Shape;
 
 /**
  * A {@link PrefixTreeStrategy} which uses {@link AbstractVisitingPrefixTreeQuery}.
@@ -121,7 +122,7 @@ public class RecursivePrefixTreeStrategy extends PrefixTreeStrategy {
 
   @Override
   protected Iterator<Cell> createCellIteratorToIndex(Shape shape, int detailLevel, Iterator<Cell> reuse) {
-    if (shape instanceof Point || !pruneLeafyBranches)
+    if (!pruneLeafyBranches || isGridAlignedShape(shape))
       return super.createCellIteratorToIndex(shape, detailLevel, reuse);
 
     List<Cell> cells = new ArrayList<>(4096);
@@ -177,6 +178,9 @@ public class RecursivePrefixTreeStrategy extends PrefixTreeStrategy {
     int detailLevel = grid.getLevelForDistance(args.resolveDistErr(ctx, distErrPct));
 
     if (op == SpatialOperation.Intersects) {
+      if (isGridAlignedShape(args.getShape())) {
+        return makeGridShapeIntersectsQuery(args.getShape());
+      }
       return new IntersectsPrefixTreeQuery(
           shape, getFieldName(), grid, detailLevel, prefixGridScanLevel);
     } else if (op == SpatialOperation.IsWithin) {
@@ -189,4 +193,35 @@ public class RecursivePrefixTreeStrategy extends PrefixTreeStrategy {
     }
     throw new UnsupportedSpatialOperation(op);
   }
+
+  /**
+   * A quick check of the shape to see if it is perfectly aligned to a grid.
+   * Points always are as they are indivisible.  It's okay to return false
+   * if the shape actually is aligned; this is an optimization hint.
+   */
+  protected boolean isGridAlignedShape(Shape shape) {
+    return isPointShape(shape);
+  }
+
+  /** {@link #makeQuery(SpatialArgs)} specialized for the query being a grid square. */
+  protected Query makeGridShapeIntersectsQuery(Shape gridShape) {
+    assert isGridAlignedShape(gridShape);
+    if (isPointsOnly()) {
+      // Awesome; this will be equivalent to a TermQuery.
+      Iterator<Cell> cellIterator = grid.getTreeCellIterator(gridShape, grid.getMaxLevels());
+      // get last cell
+      Cell cell = cellIterator.next();
+      while (cellIterator.hasNext()) {
+        int prevLevel = cell.getLevel();
+        cell = cellIterator.next();
+        assert prevLevel < cell.getLevel();
+      }
+      return new TermQuery(new Term(getFieldName(), cell.getTokenBytesWithLeaf(null)));
+    } else {
+      // Well there could be parent cells. But we can reduce the "scan level" which will be slower for a point query.
+      // TODO: AVPTQ will still scan the bottom nonetheless; file an issue to eliminate that
+      return new IntersectsPrefixTreeQuery(
+          gridShape, getFieldName(), grid, getGrid().getMaxLevels(), getGrid().getMaxLevels() + 1);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d4f87b4a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/DateNRStrategyTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/DateNRStrategyTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/DateNRStrategyTest.java
index 77c2529..54296da 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/DateNRStrategyTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/DateNRStrategyTest.java
@@ -27,7 +27,7 @@ import org.junit.Before;
 import org.junit.Test;
 import org.locationtech.spatial4j.shape.Shape;
 
-import static com.carrotsearch.randomizedtesting.RandomizedTest.randomBoolean;
+import static com.carrotsearch.randomizedtesting.RandomizedTest.randomInt;
 import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween;
 
 public class DateNRStrategyTest extends RandomSpatialOpStrategyTestCase {
@@ -42,17 +42,8 @@ public class DateNRStrategyTest extends RandomSpatialOpStrategyTestCase {
   public void setUp() throws Exception {
     super.setUp();
     tree = DateRangePrefixTree.INSTANCE;
-    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();
-        }
-      };
-    }
+    strategy = new NumberRangePrefixTreeStrategy(tree, "dateRange");
+    ((NumberRangePrefixTreeStrategy)strategy).setPointsOnly(randomInt() % 5 == 0);
     Calendar tmpCal = tree.newCal();
     int randomCalWindowField = randomIntBetween(Calendar.YEAR, Calendar.MILLISECOND);
     tmpCal.add(randomCalWindowField, 2_000);
@@ -79,15 +70,16 @@ public class DateNRStrategyTest extends RandomSpatialOpStrategyTestCase {
 
   @Test
   public void testWithinSame() throws IOException {
-    final Calendar cal = tree.newCal();
+    Shape shape = randomIndexedShape();
     testOperation(
-        tree.toShape(cal),
+        shape,
         SpatialOperation.IsWithin,
-        tree.toShape(cal), true);//is within itself
+        shape, true);//is within itself
   }
 
   @Test
   public void testWorld() throws IOException {
+    ((NumberRangePrefixTreeStrategy)strategy).setPointsOnly(false);
     testOperation(
         tree.toShape(tree.newCal()),//world matches everything
         SpatialOperation.Contains,
@@ -96,6 +88,7 @@ public class DateNRStrategyTest extends RandomSpatialOpStrategyTestCase {
 
   @Test
   public void testBugInitIterOptimization() throws Exception {
+    ((NumberRangePrefixTreeStrategy)strategy).setPointsOnly(false);
     //bug due to fast path initIter() optimization
     testOperation(
         tree.parseShape("[2014-03-27T23 TO 2014-04-01T01]"),
@@ -114,6 +107,21 @@ public class DateNRStrategyTest extends RandomSpatialOpStrategyTestCase {
 
   @Override
   protected Shape randomIndexedShape() {
+    if (((NumberRangePrefixTreeStrategy)strategy).isPointsOnly()) {
+      Calendar cal = tree.newCal();
+      cal.setTimeInMillis(random().nextLong());
+      return tree.toShape(cal);
+    } else {
+      return randomShape();
+    }
+  }
+
+  @Override
+  protected Shape randomQueryShape() {
+    return randomShape();
+  }
+
+  private Shape randomShape() {
     Calendar cal1 = randomCalendar();
     UnitNRShape s1 = tree.toShape(cal1);
     if (rarely()) {
@@ -144,9 +152,4 @@ public class DateNRStrategyTest extends RandomSpatialOpStrategyTestCase {
     }
     return cal;
   }
-
-  @Override
-  protected Shape randomQueryShape() {
-    return randomIndexedShape();
-  }
 }